Merge branch 'gerrit/stabilization-5949a1cb71'
Change-Id: I7e4e013b2c3a468dab2f8d4905dc3fd47db6b3f9
diff --git a/.gitignore b/.gitignore
index 1dcc3c2..295d874 100644
--- a/.gitignore
+++ b/.gitignore
@@ -27,5 +27,3 @@
*.swp
.m2*
ß
-
-.SpiderSilkWorkingDirectory
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java
index 4adffda..458008b 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/ActiveManager.java
@@ -38,6 +38,7 @@
import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.common.memory.ConcurrentFramePool;
import org.apache.hyracks.api.application.INCServiceContext;
+import org.apache.hyracks.api.client.NodeStatus;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.util.JavaSerializationUtils;
import org.apache.hyracks.control.nc.NodeControllerService;
@@ -72,6 +73,11 @@
}
public void registerRuntime(IActiveRuntime runtime) throws HyracksDataException {
+ NodeControllerService controllerService = (NodeControllerService) serviceCtx.getControllerService();
+ if (controllerService.getNodeStatus() != NodeStatus.ACTIVE) {
+ throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.NODE_IS_NOT_ACTIVE,
+ serviceCtx.getNodeId());
+ }
if (shutdown) {
throw new RuntimeDataException(ErrorCode.ACTIVE_MANAGER_SHUTDOWN);
}
@@ -98,7 +104,7 @@
}
public void submit(ActiveManagerMessage message) throws HyracksDataException {
- LOGGER.log(Level.INFO, "Message of type " + message.getKind() + " received in " + nodeId);
+ LOGGER.debug("Message of type {} received in {}", message.getKind(), nodeId);
switch (message.getKind()) {
case STOP_ACTIVITY:
stopRuntime(message);
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEntityEventSubscriber.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEntityEventSubscriber.java
index e01d0a7..3c2f8e8 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEntityEventSubscriber.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEntityEventSubscriber.java
@@ -34,7 +34,7 @@
void notify(ActiveEvent event);
/**
- * Checkcs whether the subscriber is done receiving events
+ * Checks whether the subscriber is done receiving events
*
* @return
*/
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEntityEventsListener.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEntityEventsListener.java
index ca610aa..8338b2b 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEntityEventsListener.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/IActiveEntityEventsListener.java
@@ -75,7 +75,7 @@
* refresh the stats
*
* @param timeout
- * @throws HyracksDataException
+ * @throws HyracksDataException throws ASX3118 if active entity is not currently running
*/
void refreshStats(long timeout) throws HyracksDataException;
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActiveManagerMessage.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActiveManagerMessage.java
index b8c44a6..1a2af13 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActiveManagerMessage.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActiveManagerMessage.java
@@ -56,6 +56,6 @@
@Override
public String toString() {
- return ActiveManagerMessage.class.getSimpleName();
+ return getClass().getSimpleName();
}
}
diff --git a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActiveStatsRequestMessage.java b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActiveStatsRequestMessage.java
index 0dbba52..117a68c 100644
--- a/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActiveStatsRequestMessage.java
+++ b/asterixdb/asterix-active/src/main/java/org/apache/asterix/active/message/ActiveStatsRequestMessage.java
@@ -29,7 +29,17 @@
this.reqId = reqId;
}
+ @Override
+ public boolean isWhispered() {
+ return true;
+ }
+
public long getReqId() {
return reqId;
}
+
+ @Override
+ public String toString() {
+ return "ActiveStatsRequestMessage{" + "reqId=" + reqId + '}';
+ }
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionStatement.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionStatement.java
index 15267aa..9507e16 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionStatement.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionStatement.java
@@ -36,6 +36,8 @@
return Kind.EXTENSION;
}
+ public abstract String getName();
+
/**
* Called when the {@code IStatementExecutor} encounters an extension statement.
* An implementation class should implement the actual processing of the statement in this method.
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
index d8fa3d6..6f359c5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
@@ -29,6 +29,7 @@
import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.NoOpWarningCollector;
import org.apache.asterix.common.exceptions.WarningCollector;
import org.apache.asterix.common.exceptions.WarningUtil;
import org.apache.asterix.dataflow.data.common.ExpressionTypeComputer;
@@ -69,6 +70,7 @@
import org.apache.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.ExpressionRuntimeProvider;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.StatefulFunctionCallExpression;
@@ -81,6 +83,7 @@
import org.apache.hyracks.algebricks.core.config.AlgebricksConfig;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -143,7 +146,8 @@
BinaryComparatorFactoryProvider.INSTANCE, TypeTraitProvider.INSTANCE, BinaryBooleanInspector.FACTORY,
BinaryIntegerInspector.FACTORY, ADMPrinterFactoryProvider.INSTANCE, MissingWriterFactory.INSTANCE, null,
new ExpressionRuntimeProvider(new QueryLogicalExpressionJobGen(metadataProvider.getFunctionManager())),
- ExpressionTypeComputer.INSTANCE, null, null, null, null, GlobalConfig.DEFAULT_FRAME_SIZE, null, 0);
+ ExpressionTypeComputer.INSTANCE, null, null, null, null, GlobalConfig.DEFAULT_FRAME_SIZE, null,
+ NoOpWarningCollector.INSTANCE, 0, new PhysicalOptimizationConfig());
}
@Override
@@ -202,7 +206,21 @@
public Pair<Boolean, ILogicalExpression> visitScalarFunctionCallExpression(ScalarFunctionCallExpression expr,
Void arg) throws AlgebricksException {
boolean changed = constantFoldArgs(expr, arg);
- if (!allArgsConstant(expr) || !expr.isFunctional() || !canConstantFold(expr)) {
+ List<Mutable<ILogicalExpression>> argList = expr.getArguments();
+ int argConstantCount = countConstantArgs(argList);
+ if (argConstantCount != argList.size()) {
+ if (argConstantCount > 0 && expr.getFunctionIdentifier().equals(BuiltinFunctions.OR)
+ && expr.isFunctional()) {
+ if (foldOrArgs(expr)) {
+ ILogicalExpression changedExpr =
+ expr.getArguments().size() == 1 ? expr.getArguments().get(0).getValue() : expr;
+ return new Pair<>(true, changedExpr);
+ }
+ }
+ return new Pair<>(changed, expr);
+ }
+
+ if (!expr.isFunctional() || !canConstantFold(expr)) {
return new Pair<>(changed, expr);
}
@@ -346,13 +364,14 @@
return false;
}
- private boolean allArgsConstant(AbstractFunctionCallExpression expr) {
- for (Mutable<ILogicalExpression> r : expr.getArguments()) {
- if (r.getValue().getExpressionTag() != LogicalExpressionTag.CONSTANT) {
- return false;
+ private int countConstantArgs(List<Mutable<ILogicalExpression>> argList) {
+ int n = 0;
+ for (Mutable<ILogicalExpression> r : argList) {
+ if (r.getValue().getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+ n++;
}
}
- return true;
+ return n;
}
private boolean canConstantFold(ScalarFunctionCallExpression function) throws AlgebricksException {
@@ -389,5 +408,37 @@
}
return true;
}
+
+ private boolean foldOrArgs(ScalarFunctionCallExpression expr) {
+ // or(true,x,y) -> true; or(false,x,y) -> or(x,y)
+ boolean changed = false;
+ List<Mutable<ILogicalExpression>> argList = expr.getArguments();
+ Iterator<Mutable<ILogicalExpression>> argIter = argList.iterator();
+ Mutable<ILogicalExpression> argFalse = null;
+ while (argIter.hasNext()) {
+ Mutable<ILogicalExpression> argExprRef = argIter.next();
+ ILogicalExpression argExpr = argExprRef.getValue();
+ if (argExpr.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+ continue;
+ }
+ ConstantExpression cExpr = (ConstantExpression) argExpr;
+ IAlgebricksConstantValue cValue = cExpr.getValue();
+ if (cValue.isTrue()) {
+ // or(true,x,y) -> true;
+ argList.clear();
+ argList.add(argExprRef);
+ return true;
+ } else if (cValue.isFalse()) {
+ // remove 'false' from arg list, but save the expression.
+ argFalse = argExprRef;
+ argIter.remove();
+ changed = true;
+ }
+ }
+ if (argList.isEmpty() && argFalse != null) {
+ argList.add(argFalse);
+ }
+ return changed;
+ }
}
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/DisjunctivePredicateToJoinRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/DisjunctivePredicateToJoinRule.java
index 916fd75..fe03d75 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/DisjunctivePredicateToJoinRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/DisjunctivePredicateToJoinRule.java
@@ -18,9 +18,13 @@
*/
package org.apache.asterix.optimizer.rules;
+import java.util.Collections;
+import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
+import java.util.Map;
+import org.apache.asterix.common.annotations.SkipSecondaryIndexSearchExpressionAnnotation;
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.om.base.AOrderedList;
import org.apache.asterix.om.constants.AsterixConstantValue;
@@ -39,6 +43,7 @@
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.BroadcastExpressionAnnotation;
import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
import org.apache.hyracks.algebricks.core.algebra.expressions.IndexedNLJoinExpressionAnnotation;
import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
@@ -89,6 +94,7 @@
VariableReferenceExpression varEx = null;
IAType valType = null;
HashSet<AsterixConstantValue> values = new HashSet<AsterixConstantValue>();
+ Map<Object, IExpressionAnnotation> allAnnotations = Collections.emptyMap();
for (Mutable<ILogicalExpression> arg : args) {
AbstractFunctionCallExpression fctCall;
@@ -128,6 +134,12 @@
if (!(haveVar && haveConst)) {
return false;
}
+ if (!fctCall.getAnnotations().isEmpty()) {
+ if (allAnnotations.isEmpty()) {
+ allAnnotations = new HashMap<>();
+ }
+ allAnnotations.putAll(fctCall.getAnnotations());
+ }
}
SourceLocation sourceLoc = select.getSourceLocation();
@@ -142,8 +154,9 @@
ILogicalExpression cExp = new ConstantExpression(new AsterixConstantValue(list));
Mutable<ILogicalExpression> mutCExp = new MutableObject<>(cExp);
- IFunctionInfo scanFctInfo = BuiltinFunctions.getAsterixFunctionInfo(BuiltinFunctions.SCAN_COLLECTION);
- UnnestingFunctionCallExpression scanExp = new UnnestingFunctionCallExpression(scanFctInfo, mutCExp);
+ UnnestingFunctionCallExpression scanExp = new UnnestingFunctionCallExpression(
+ BuiltinFunctions.getAsterixFunctionInfo(BuiltinFunctions.SCAN_COLLECTION));
+ scanExp.getArguments().add(mutCExp);
scanExp.setSourceLocation(sourceLoc);
LogicalVariable scanVar = context.newVar();
UnnestOperator unn = new UnnestOperator(scanVar, new MutableObject<>(scanExp));
@@ -158,11 +171,14 @@
scanVarRef.setSourceLocation(sourceLoc);
eqExp.getArguments().add(new MutableObject<>(scanVarRef));
eqExp.getArguments().add(new MutableObject<>(varEx.cloneExpression()));
- eqExp.getAnnotations().put(IndexedNLJoinExpressionAnnotation.INSTANCE,
- IndexedNLJoinExpressionAnnotation.INSTANCE);
+ if (!allAnnotations.containsKey(SkipSecondaryIndexSearchExpressionAnnotation.INSTANCE)) {
+ eqExp.getAnnotations().put(IndexedNLJoinExpressionAnnotation.INSTANCE,
+ IndexedNLJoinExpressionAnnotation.INSTANCE);
+ }
BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
bcast.setObject(BroadcastExpressionAnnotation.BroadcastSide.LEFT); // Broadcast the OR predicates branch.
eqExp.getAnnotations().put(bcast, bcast);
+ eqExp.getAnnotations().putAll(allAnnotations);
InnerJoinOperator jOp = new InnerJoinOperator(new MutableObject<>(eqExp));
jOp.setSourceLocation(sourceLoc);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyJoinRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyJoinRule.java
index cc91c06..be49396 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyJoinRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FuzzyJoinRule.java
@@ -424,7 +424,7 @@
try {
clauses = parser.Clauses();
} catch (ParseException e) {
- throw CompilationException.create(ErrorCode.COMPILATION_TRANSLATION_ERROR, e);
+ throw CompilationException.create(ErrorCode.COMPILATION_TRANSLATION_ERROR, String.valueOf(e), e);
}
// Step 4. The essential substitution with translator.
@@ -432,7 +432,7 @@
try {
plan = translator.translate(clauses);
} catch (CompilationException e) {
- throw CompilationException.create(ErrorCode.COMPILATION_TRANSLATION_ERROR, e);
+ throw CompilationException.create(ErrorCode.COMPILATION_TRANSLATION_ERROR, String.valueOf(e), e);
}
context.setVarCounter(counter.get());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceEnforcedListTypeRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceEnforcedListTypeRule.java
index 72f24b0..723fff2 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceEnforcedListTypeRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceEnforcedListTypeRule.java
@@ -64,6 +64,7 @@
* rewrite list constructor types for list constructor functions
*/
List<Mutable<ILogicalExpression>> expressions;
+ // TODO(ali): what about other operators that could have list expressions?
switch (op.getOperatorTag()) {
case ASSIGN:
AbstractAssignOperator assignOp = (AbstractAssignOperator) op;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
index 6c258e4..37b6ca7 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
@@ -75,6 +75,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteUpsertOperator.Kind;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOperator;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
import org.apache.hyracks.api.exceptions.SourceLocation;
@@ -350,31 +351,40 @@
// TokenizeOperator to tokenize [SK, PK] pairs
TokenizeOperator tokenUpdate = new TokenizeOperator(dataSourceIndex,
- primaryIndexModificationOp.getPrimaryKeyExpressions(), secondaryExpressions,
- tokenizeKeyVars, filterExpression, primaryIndexModificationOp.getOperation(),
- primaryIndexModificationOp.isBulkload(), isPartitioned, varTypes);
+ OperatorManipulationUtil
+ .cloneExpressions(primaryIndexModificationOp.getPrimaryKeyExpressions()),
+ secondaryExpressions, tokenizeKeyVars,
+ filterExpression != null
+ ? new MutableObject<>(filterExpression.getValue().cloneExpression()) : null,
+ primaryIndexModificationOp.getOperation(), primaryIndexModificationOp.isBulkload(),
+ isPartitioned, varTypes);
tokenUpdate.setSourceLocation(sourceLoc);
tokenUpdate.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
context.computeAndSetTypeEnvironmentForOperator(tokenUpdate);
replicateOutput = tokenUpdate;
indexUpdate = new IndexInsertDeleteUpsertOperator(dataSourceIndex,
- primaryIndexModificationOp.getPrimaryKeyExpressions(), tokenizeKeyExprs, filterExpression,
- primaryIndexModificationOp.getOperation(), primaryIndexModificationOp.isBulkload(),
- primaryIndexModificationOp.getAdditionalNonFilteringExpressions() == null ? 0
- : primaryIndexModificationOp.getAdditionalNonFilteringExpressions().size());
- indexUpdate.setSourceLocation(sourceLoc);
- indexUpdate.setAdditionalFilteringExpressions(filteringExpressions);
- indexUpdate.getInputs().add(new MutableObject<ILogicalOperator>(tokenUpdate));
- } else {
- // When TokenizeOperator is not needed
- indexUpdate = new IndexInsertDeleteUpsertOperator(dataSourceIndex,
- primaryIndexModificationOp.getPrimaryKeyExpressions(), secondaryExpressions,
- filterExpression, primaryIndexModificationOp.getOperation(),
+ OperatorManipulationUtil
+ .cloneExpressions(primaryIndexModificationOp.getPrimaryKeyExpressions()),
+ tokenizeKeyExprs, filterExpression, primaryIndexModificationOp.getOperation(),
primaryIndexModificationOp.isBulkload(),
primaryIndexModificationOp.getAdditionalNonFilteringExpressions() == null ? 0
: primaryIndexModificationOp.getAdditionalNonFilteringExpressions().size());
indexUpdate.setSourceLocation(sourceLoc);
- indexUpdate.setAdditionalFilteringExpressions(filteringExpressions);
+ indexUpdate.setAdditionalFilteringExpressions(
+ OperatorManipulationUtil.cloneExpressions(filteringExpressions));
+ indexUpdate.getInputs().add(new MutableObject<ILogicalOperator>(tokenUpdate));
+ } else {
+ // When TokenizeOperator is not needed
+ indexUpdate = new IndexInsertDeleteUpsertOperator(dataSourceIndex,
+ OperatorManipulationUtil
+ .cloneExpressions(primaryIndexModificationOp.getPrimaryKeyExpressions()),
+ secondaryExpressions, filterExpression, primaryIndexModificationOp.getOperation(),
+ primaryIndexModificationOp.isBulkload(),
+ primaryIndexModificationOp.getAdditionalNonFilteringExpressions() == null ? 0
+ : primaryIndexModificationOp.getAdditionalNonFilteringExpressions().size());
+ indexUpdate.setSourceLocation(sourceLoc);
+ indexUpdate.setAdditionalFilteringExpressions(
+ OperatorManipulationUtil.cloneExpressions(filteringExpressions));
replicateOutput = indexUpdate;
// We add the necessary expressions for upsert
if (primaryIndexModificationOp.getOperation() == Kind.UPSERT) {
@@ -478,12 +488,15 @@
}
DataSourceIndex dataSourceIndex = new DataSourceIndex(index, dataverseName, datasetName, mp);
indexUpdate = new IndexInsertDeleteUpsertOperator(dataSourceIndex,
- primaryIndexModificationOp.getPrimaryKeyExpressions(), secondaryExpressions, filterExpression,
- primaryIndexModificationOp.getOperation(), primaryIndexModificationOp.isBulkload(),
+ OperatorManipulationUtil
+ .cloneExpressions(primaryIndexModificationOp.getPrimaryKeyExpressions()),
+ secondaryExpressions, filterExpression, primaryIndexModificationOp.getOperation(),
+ primaryIndexModificationOp.isBulkload(),
primaryIndexModificationOp.getAdditionalNonFilteringExpressions() == null ? 0
: primaryIndexModificationOp.getAdditionalNonFilteringExpressions().size());
indexUpdate.setSourceLocation(sourceLoc);
- indexUpdate.setAdditionalFilteringExpressions(filteringExpressions);
+ indexUpdate.setAdditionalFilteringExpressions(
+ OperatorManipulationUtil.cloneExpressions(filteringExpressions));
if (primaryIndexModificationOp.getOperation() == Kind.UPSERT) {
// set before op secondary key expressions
if (filteringFields != null) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/LoadRecordFieldsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/LoadRecordFieldsRule.java
index 6762e77..b9d512b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/LoadRecordFieldsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/LoadRecordFieldsRule.java
@@ -292,7 +292,7 @@
VariableUtilities.getLiveVariables(assign, liveInputVars);
usedVariables.removeAll(liveInputVars);
if (usedVariables.isEmpty()) {
- assign.getExpressions().get(0).setValue(fldExpr);
+ assign.getExpressions().get(0).setValue(fldExpr.cloneExpression());
return true;
} else {
return false;
@@ -309,7 +309,17 @@
private static ILogicalExpression findFieldExpression(AbstractLogicalOperator op, LogicalVariable recordVar,
Object accessKey, IVariableTypeEnvironment typeEnvironment, FieldResolver resolver)
throws AlgebricksException {
- for (Mutable<ILogicalOperator> child : op.getInputs()) {
+ List<Mutable<ILogicalOperator>> inputs = op.getInputs();
+ int inputIdxLimit;
+ if (op.getOperatorTag() == LogicalOperatorTag.LEFTOUTERJOIN) {
+ // do not search in the right branch of a left outer join
+ inputIdxLimit = 1;
+ } else {
+ inputIdxLimit = inputs.size();
+ }
+
+ for (int inputIdx = 0; inputIdx < inputIdxLimit; inputIdx++) {
+ Mutable<ILogicalOperator> child = inputs.get(inputIdx);
AbstractLogicalOperator opChild = (AbstractLogicalOperator) child.getValue();
if (opChild.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
AssignOperator op2 = (AssignOperator) opChild;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushGroupByThroughProduct.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushGroupByThroughProduct.java
index 48c4324..6d92b51 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushGroupByThroughProduct.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushGroupByThroughProduct.java
@@ -84,24 +84,27 @@
Mutable<ILogicalOperator> opLeftRef = join.getInputs().get(0);
ILogicalOperator opLeft = opLeftRef.getValue();
+ Mutable<ILogicalOperator> opRightRef = join.getInputs().get(1);
+ ILogicalOperator opRight = opRightRef.getValue();
switch (canPushThrough(gby, opLeft, decorToPush, decorNotToPush)) {
case REPEATED_DECORS: {
return false;
}
case TRUE: {
- push(opRef, opRef2, 0, decorToPush, decorNotToPush, context);
- return true;
+ if (OperatorPropertiesUtil.isCardinalityZeroOrOne(opRight)) {
+ push(opRef, opRef2, 0, decorToPush, decorNotToPush, context);
+ return true;
+ }
+ return false;
}
case FALSE: {
decorToPush.clear();
- Mutable<ILogicalOperator> opRightRef = join.getInputs().get(1);
- ILogicalOperator opRight = opRightRef.getValue();
- if (canPushThrough(gby, opRight, decorToPush, decorNotToPush) == PushTestResult.TRUE) {
+ if (canPushThrough(gby, opRight, decorToPush, decorNotToPush) == PushTestResult.TRUE
+ && OperatorPropertiesUtil.isCardinalityZeroOrOne(opLeft)) {
push(opRef, opRef2, 1, decorToPush, decorNotToPush, context);
return true;
- } else {
- return false;
}
+ return false;
}
default: {
throw new IllegalStateException();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveLeftOuterUnnestForLeftOuterJoinRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveLeftOuterUnnestForLeftOuterJoinRule.java
index 364816b..775a1df 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveLeftOuterUnnestForLeftOuterJoinRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveLeftOuterUnnestForLeftOuterJoinRule.java
@@ -91,7 +91,8 @@
Triple<Boolean, ILogicalExpression, ILogicalExpression> checkGbyResult =
checkUnnestAndGby(outerUnnest, gbyOperator);
// The argument for listify and not(is-missing(...)) check should be variables.
- if (!isVariableReference(checkGbyResult.second) || !isVariableReference(checkGbyResult.third)) {
+ if (!checkGbyResult.first || checkGbyResult.second == null || !isVariableReference(checkGbyResult.second)
+ || checkGbyResult.third == null || !isVariableReference(checkGbyResult.third)) {
return false;
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
index 4b6fc36..87a2d03 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/BTreeAccessMethod.java
@@ -570,8 +570,9 @@
if (lowKeyLimits[0] == null && lowKeyLimits[i] != null || lowKeyLimits[0] != null && lowKeyLimits[i] == null
|| highKeyLimits[0] == null && highKeyLimits[i] != null
|| highKeyLimits[0] != null && highKeyLimits[i] == null) {
- numSecondaryKeys--;
+ numSecondaryKeys = i;
primaryIndexPostProccessingIsNeeded = true;
+ break;
}
}
@@ -791,6 +792,9 @@
} else {
keyVar = ((VariableReferenceExpression) searchKeyExpr).getVariableReference();
if (constExpression != null) {
+ if (constExpression.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+ constExpression = constExpression.cloneExpression();
+ }
assignKeyExprList.add(new MutableObject<>(constExpression));
assignKeyVarList.add(constExprVars[i]);
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/AsterixMoveFreeVariableOperatorOutOfSubplanRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/AsterixMoveFreeVariableOperatorOutOfSubplanRule.java
index 4d6b4bb..66dcc83 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/AsterixMoveFreeVariableOperatorOutOfSubplanRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/AsterixMoveFreeVariableOperatorOutOfSubplanRule.java
@@ -18,13 +18,22 @@
*/
package org.apache.asterix.optimizer.rules.subplan;
+import java.util.Set;
+
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.rewriter.rules.subplan.MoveFreeVariableOperatorOutOfSubplanRule;
public class AsterixMoveFreeVariableOperatorOutOfSubplanRule extends MoveFreeVariableOperatorOutOfSubplanRule {
@Override
- protected boolean movableOperator(LogicalOperatorTag operatorTag) {
+ protected boolean movableOperatorKind(LogicalOperatorTag operatorTag) {
return (operatorTag == LogicalOperatorTag.ASSIGN);
}
+
+ @Override
+ protected boolean movableIndependentOperator(ILogicalOperator op, Set<LogicalVariable> usedVars) {
+ return !usedVars.isEmpty();
+ }
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
index 78c4c5e..02dc6f6 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
@@ -626,10 +626,11 @@
@Override
public ILogicalOperator visitDistinctOperator(DistinctOperator op, Void arg) throws AlgebricksException {
visitSingleInputOperator(op);
- List<LogicalVariable> distinctVarList = op.getDistinctByVarList();
for (LogicalVariable keyVar : correlatedKeyVars) {
- if (!distinctVarList.contains(keyVar)) {
- distinctVarList.add(keyVar);
+ if (!op.isDistinctByVar(keyVar)) {
+ VariableReferenceExpression keyVarRef = new VariableReferenceExpression(keyVar);
+ keyVarRef.setSourceLocation(op.getSourceLocation());
+ op.getExpressions().add(new MutableObject<>(keyVarRef));
}
}
context.computeAndSetTypeEnvironmentForOperator(op);
@@ -654,7 +655,17 @@
@Override
public ILogicalOperator visitWindowOperator(WindowOperator op, Void arg) throws AlgebricksException {
- return visitSingleInputOperator(op);
+ visitSingleInputOperator(op);
+ List<LogicalVariable> partitionByVars = op.getPartitionVarList();
+ for (LogicalVariable keyVar : correlatedKeyVars) {
+ if (!partitionByVars.contains(keyVar)) {
+ VariableReferenceExpression keyVarRef = new VariableReferenceExpression(keyVar);
+ keyVarRef.setSourceLocation(op.getSourceLocation());
+ op.getPartitionExpressions().add(new MutableObject<>(keyVarRef));
+ }
+ }
+ context.computeAndSetTypeEnvironmentForOperator(op);
+ return op;
}
/**
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
index 12596ff..8326f85 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
@@ -361,10 +361,11 @@
if (!rewritten || !underJoin) {
return op;
}
- List<LogicalVariable> distinctVarList = op.getDistinctByVarList();
for (LogicalVariable keyVar : liveVarsFromSubplanInput) {
- if (!distinctVarList.contains(keyVar)) {
- distinctVarList.add(keyVar);
+ if (!op.isDistinctByVar(keyVar)) {
+ VariableReferenceExpression keyVarRef = new VariableReferenceExpression(keyVar);
+ keyVarRef.setSourceLocation(op.getSourceLocation());
+ op.getExpressions().add(new MutableObject<>(keyVarRef));
}
}
context.computeAndSetTypeEnvironmentForOperator(op);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java
index 288b01a..a0bb4b6 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java
@@ -220,7 +220,7 @@
@Override
public Boolean visitDistinctOperator(DistinctOperator op, Void arg) throws AlgebricksException {
- return visitInputs(op);
+ return visitTupleDiscardingOperator(op);
}
@Override
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
index ac2460e..e96a192 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
@@ -69,7 +69,7 @@
*
* @author yingyib
*/
-public class StaticTypeCastUtil {
+public final class StaticTypeCastUtil {
private StaticTypeCastUtil() {
}
@@ -82,10 +82,11 @@
* rule: @ IntroduceEnforcedListTypeRule.
*
* @param funcExpr
- * record constructor function expression
+ * a function expression
* @param reqType
- * required record type
+ * required (list) type (for when the funcExpr is a list constructor)
* @param inputType
+ * inferred (list) type (for when the funcExpr is a list constructor)
* @param env
* type environment
* @throws AlgebricksException
@@ -136,7 +137,7 @@
* @param reqType
* the required type inferred from parent operators/expressions
* @param inputType
- * the current inferred
+ * the current inferred type
* @param env
* the type environment
* @return true if the type is casted; otherwise, false.
@@ -162,6 +163,7 @@
return rewriteListFuncExpr(funcExpr, (AbstractCollectionType) reqType, (AbstractCollectionType) inputType,
env);
} else if (inputType.getTypeTag().equals(ATypeTag.OBJECT)) {
+ // TODO(ali): inputType? shouldn't we check against the funcExpr whether it's a record constructor?
if (reqType.equals(BuiltinType.ANY)) {
reqType = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
}
@@ -186,13 +188,14 @@
}
/**
- * only called when funcExpr is record constructor
+ * Only called when funcExpr's inferred type (the inputRecordType) is a record.
*
* @param funcExpr
- * record constructor function expression
+ * a function expression that produces a record
* @param requiredRecordType
* required record type
* @param inputRecordType
+ * inferred record type of funcExpr
* @param env
* type environment
* @throws AlgebricksException
@@ -213,13 +216,14 @@
}
/**
- * only called when funcExpr is list constructor
+ * Only called when funcExpr is list constructor.
*
* @param funcExpr
* list constructor function expression
* @param requiredListType
* required list type
* @param inputListType
+ * inferred list type
* @param env
* type environment
* @throws AlgebricksException
@@ -247,19 +251,22 @@
case FUNCTION_CALL:
ScalarFunctionCallExpression argFunc = (ScalarFunctionCallExpression) arg;
changed |= rewriteFuncExpr(argFunc, requiredItemType, currentItemType, env);
- changed |= castItem(argRef, argFunc, requiredItemType, env);
+ changed |= castItem(argRef, requiredItemType, env);
break;
case VARIABLE:
- // TODO(ali): why are we always casting to an open type without considering "requiredItemType"?
- changed |= injectCastToRelaxType(argRef, currentItemType, env);
+ changed |= castItem(argRef, requiredItemType, env);
+ break;
+ case CONSTANT:
+ // TODO(ali): should the constant be handled (i.e. constant array or record)?
break;
}
}
return changed;
}
- private static boolean castItem(Mutable<ILogicalExpression> itemExprRef, ScalarFunctionCallExpression itemExpr,
- IAType requiredItemType, IVariableTypeEnvironment env) throws AlgebricksException {
+ private static boolean castItem(Mutable<ILogicalExpression> itemExprRef, IAType requiredItemType,
+ IVariableTypeEnvironment env) throws AlgebricksException {
+ ILogicalExpression itemExpr = itemExprRef.getValue();
IAType itemType = (IAType) env.getType(itemExpr);
if (TypeResolverUtil.needsCast(requiredItemType, itemType) && !satisfied(requiredItemType, itemType)) {
injectCastFunction(FunctionUtil.getFunctionInfo(BuiltinFunctions.CAST_TYPE), requiredItemType, itemType,
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IRequestParameters.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IRequestParameters.java
index e242258..417a130 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IRequestParameters.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IRequestParameters.java
@@ -22,6 +22,7 @@
import org.apache.asterix.common.api.ICommonRequestParameters;
import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.translator.IStatementExecutor.StatementProperties;
import org.apache.asterix.translator.IStatementExecutor.Stats;
import org.apache.hyracks.api.result.IResultSet;
@@ -45,6 +46,12 @@
Stats getStats();
/**
+ * @return a reference on which to write properties of executed queries (e.g. what kind of statement was parsed
+ * by the parser)
+ */
+ StatementProperties getStatementProperties();
+
+ /**
* @return a reference to write the metadata of executed queries
*/
IStatementExecutor.ResultMetadata getOutMetadata();
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
index e7fec0c..5911c1e 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
@@ -36,6 +36,7 @@
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.lang.common.base.IStatementRewriter;
+import org.apache.asterix.lang.common.base.Statement;
import org.apache.asterix.lang.common.statement.Query;
import org.apache.asterix.metadata.declared.MetadataProvider;
import org.apache.asterix.om.base.IAObject;
@@ -216,6 +217,38 @@
}
}
+ class StatementProperties implements Serializable {
+ private static final long serialVersionUID = -1L;
+
+ private Statement.Kind kind;
+ private String name;
+
+ public Statement.Kind getKind() {
+ return kind;
+ }
+
+ public void setKind(Statement.Kind kind) {
+ this.kind = kind;
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ public void setName(String name) {
+ this.name = name;
+ }
+
+ public boolean isValid() {
+ return kind != null && (kind != Statement.Kind.EXTENSION || name != null);
+ }
+
+ @Override
+ public String toString() {
+ return Statement.Kind.EXTENSION == kind ? String.valueOf(name) : String.valueOf(kind);
+ }
+ }
+
/**
* Compiles and executes a list of statements
*
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index 5851467..e8ceba4 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -413,22 +413,19 @@
assign.getInputs().add(new MutableObject<>(topOp));
}
- VariableReferenceExpression resVarRef2 = new VariableReferenceExpression(resVar);
- resVarRef2.setSourceLocation(sourceLoc);
- Mutable<ILogicalExpression> varRef = new MutableObject<>(resVarRef2);
ILogicalOperator leafOperator;
switch (stmt.getKind()) {
case INSERT:
- leafOperator = translateInsert(targetDatasource, varRef, varRefsForLoading,
+ leafOperator = translateInsert(targetDatasource, resVar, varRefsForLoading,
additionalFilteringExpressions, assign, stmt, resultMetadata);
break;
case UPSERT:
- leafOperator = translateUpsert(targetDatasource, varRef, varRefsForLoading,
+ leafOperator = translateUpsert(targetDatasource, resVar, varRefsForLoading,
additionalFilteringExpressions, assign, additionalFilteringField, unnestVar, topOp, exprs,
- resVar, additionalFilteringAssign, stmt, resultMetadata);
+ additionalFilteringAssign, stmt, resultMetadata);
break;
case DELETE:
- leafOperator = translateDelete(targetDatasource, varRef, varRefsForLoading,
+ leafOperator = translateDelete(targetDatasource, resVar, varRefsForLoading,
additionalFilteringExpressions, assign, stmt);
break;
default:
@@ -443,7 +440,7 @@
return plan;
}
- private ILogicalOperator translateDelete(DatasetDataSource targetDatasource, Mutable<ILogicalExpression> varRef,
+ private ILogicalOperator translateDelete(DatasetDataSource targetDatasource, LogicalVariable resVar,
List<Mutable<ILogicalExpression>> varRefsForLoading,
List<Mutable<ILogicalExpression>> additionalFilteringExpressions, ILogicalOperator assign,
ICompiledDmlStatement stmt) throws AlgebricksException {
@@ -453,8 +450,10 @@
targetDatasource.getDataset().getDatasetName()
+ ": delete from dataset is not supported on Datasets with Meta records");
}
- InsertDeleteUpsertOperator deleteOp = new InsertDeleteUpsertOperator(targetDatasource, varRef,
- varRefsForLoading, InsertDeleteUpsertOperator.Kind.DELETE, false);
+ VariableReferenceExpression varRef = new VariableReferenceExpression(resVar);
+ varRef.setSourceLocation(stmt.getSourceLocation());
+ InsertDeleteUpsertOperator deleteOp = new InsertDeleteUpsertOperator(targetDatasource,
+ new MutableObject<>(varRef), varRefsForLoading, InsertDeleteUpsertOperator.Kind.DELETE, false);
deleteOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
deleteOp.getInputs().add(new MutableObject<>(assign));
deleteOp.setSourceLocation(sourceLoc);
@@ -464,11 +463,11 @@
return leafOperator;
}
- private ILogicalOperator translateUpsert(DatasetDataSource targetDatasource, Mutable<ILogicalExpression> varRef,
+ private ILogicalOperator translateUpsert(DatasetDataSource targetDatasource, LogicalVariable resVar,
List<Mutable<ILogicalExpression>> varRefsForLoading,
List<Mutable<ILogicalExpression>> additionalFilteringExpressions, ILogicalOperator assign,
List<String> additionalFilteringField, LogicalVariable unnestVar, ILogicalOperator topOp,
- List<Mutable<ILogicalExpression>> exprs, LogicalVariable resVar, AssignOperator additionalFilteringAssign,
+ List<Mutable<ILogicalExpression>> exprs, AssignOperator additionalFilteringAssign,
ICompiledDmlStatement stmt, IResultMetadata resultMetadata) throws AlgebricksException {
SourceLocation sourceLoc = stmt.getSourceLocation();
if (!targetDatasource.getDataset().allow(topOp, DatasetUtil.OP_UPSERT)) {
@@ -521,8 +520,10 @@
}
}
// A change feed, we don't need the assign to access PKs
- upsertOp = new InsertDeleteUpsertOperator(targetDatasource, varRef, varRefsForLoading, metaExpSingletonList,
- InsertDeleteUpsertOperator.Kind.UPSERT, false);
+ VariableReferenceExpression varRef = new VariableReferenceExpression(resVar);
+ varRef.setSourceLocation(stmt.getSourceLocation());
+ upsertOp = new InsertDeleteUpsertOperator(targetDatasource, new MutableObject<>(varRef), varRefsForLoading,
+ metaExpSingletonList, InsertDeleteUpsertOperator.Kind.UPSERT, false);
upsertOp.setUpsertIndicatorVar(context.newVar());
upsertOp.setUpsertIndicatorVarType(BuiltinType.ABOOLEAN);
// Create and add a new variable used for representing the original record
@@ -554,7 +555,9 @@
topOp.getInputs().set(0, new MutableObject<>(metaAndKeysAssign));
upsertOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
} else {
- upsertOp = new InsertDeleteUpsertOperator(targetDatasource, varRef, varRefsForLoading,
+ VariableReferenceExpression varRef = new VariableReferenceExpression(resVar);
+ varRef.setSourceLocation(stmt.getSourceLocation());
+ upsertOp = new InsertDeleteUpsertOperator(targetDatasource, new MutableObject<>(varRef), varRefsForLoading,
InsertDeleteUpsertOperator.Kind.UPSERT, false);
upsertOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
upsertOp.getInputs().add(new MutableObject<>(assign));
@@ -579,7 +582,7 @@
return processReturningExpression(rootOperator, upsertOp, compiledUpsert, resultMetadata);
}
- private ILogicalOperator translateInsert(DatasetDataSource targetDatasource, Mutable<ILogicalExpression> varRef,
+ private ILogicalOperator translateInsert(DatasetDataSource targetDatasource, LogicalVariable resVar,
List<Mutable<ILogicalExpression>> varRefsForLoading,
List<Mutable<ILogicalExpression>> additionalFilteringExpressions, ILogicalOperator assign,
ICompiledDmlStatement stmt, IResultMetadata resultMetadata) throws AlgebricksException {
@@ -590,8 +593,10 @@
+ ": insert into dataset is not supported on Datasets with Meta records");
}
// Adds the insert operator.
- InsertDeleteUpsertOperator insertOp = new InsertDeleteUpsertOperator(targetDatasource, varRef,
- varRefsForLoading, InsertDeleteUpsertOperator.Kind.INSERT, false);
+ VariableReferenceExpression varRef = new VariableReferenceExpression(resVar);
+ varRef.setSourceLocation(stmt.getSourceLocation());
+ InsertDeleteUpsertOperator insertOp = new InsertDeleteUpsertOperator(targetDatasource,
+ new MutableObject<>(varRef), varRefsForLoading, InsertDeleteUpsertOperator.Kind.INSERT, false);
insertOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
insertOp.getInputs().add(new MutableObject<>(assign));
insertOp.setSourceLocation(sourceLoc);
@@ -620,8 +625,8 @@
//Create an assign operator that makes the variable used by the return expression
LogicalVariable insertedVar = context.newVar();
- AssignOperator insertedVarAssignOperator =
- new AssignOperator(insertedVar, new MutableObject<>(insertOp.getPayloadExpression().getValue()));
+ AssignOperator insertedVarAssignOperator = new AssignOperator(insertedVar,
+ new MutableObject<>(insertOp.getPayloadExpression().getValue().cloneExpression()));
insertedVarAssignOperator.getInputs().add(insertOp.getInputs().get(0));
insertedVarAssignOperator.setSourceLocation(sourceLoc);
insertOp.getInputs().set(0, new MutableObject<>(insertedVarAssignOperator));
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
index 82dc344..2cd402f 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
@@ -109,6 +109,7 @@
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
@@ -149,7 +150,7 @@
public static final String REWRITE_IN_AS_OR_OPTION = "rewrite_in_as_or";
private static final boolean REWRITE_IN_AS_OR_OPTION_DEFAULT = true;
- private Deque<Mutable<ILogicalOperator>> uncorrelatedLeftBranchStack = new ArrayDeque<>();
+ private Deque<Mutable<ILogicalOperator>> uncorrelatedRightBranchStack = new ArrayDeque<>();
private final Map<VarIdentifier, IAObject> externalVars;
private final boolean translateInAsOr;
@@ -297,10 +298,12 @@
throws CompilationException {
Mutable<ILogicalOperator> inputSrc = arg;
Pair<ILogicalOperator, LogicalVariable> topUnnest = null;
+ uncorrelatedRightBranchStack.push(inputSrc);
for (FromTerm fromTerm : fromClause.getFromTerms()) {
topUnnest = fromTerm.accept(this, inputSrc);
inputSrc = new MutableObject<>(topUnnest.first);
}
+ uncorrelatedRightBranchStack.pop();
return topUnnest;
}
@@ -328,27 +331,20 @@
Mutable<ILogicalOperator> topOpRef = new MutableObject<>(unnestOp);
if (fromTerm.hasCorrelateClauses()) {
for (AbstractBinaryCorrelateClause correlateClause : fromTerm.getCorrelateClauses()) {
- if (correlateClause.getClauseType() == ClauseType.UNNEST_CLAUSE) {
- // Correlation is allowed.
- topOpRef = new MutableObject<>(correlateClause.accept(this, topOpRef).first);
- } else {
- // Correlation is dis-allowed.
- uncorrelatedLeftBranchStack.push(topOpRef);
- topOpRef = new MutableObject<>(correlateClause.accept(this, tupSource).first);
- }
+ topOpRef = new MutableObject<>(correlateClause.accept(this, topOpRef).first);
}
}
return new Pair<>(topOpRef.getValue(), fromVar);
}
@Override
- public Pair<ILogicalOperator, LogicalVariable> visit(JoinClause joinClause, Mutable<ILogicalOperator> inputRef)
+ public Pair<ILogicalOperator, LogicalVariable> visit(JoinClause joinClause, Mutable<ILogicalOperator> leftInputRef)
throws CompilationException {
SourceLocation sourceLoc = joinClause.getSourceLocation();
- Mutable<ILogicalOperator> leftInputRef = uncorrelatedLeftBranchStack.pop();
if (joinClause.getJoinType() == JoinType.INNER) {
+ Mutable<ILogicalOperator> rightInputRef = uncorrelatedRightBranchStack.peek();
Pair<ILogicalOperator, LogicalVariable> rightBranch =
- generateUnnestForBinaryCorrelateRightBranch(joinClause, inputRef, true);
+ generateUnnestForBinaryCorrelateRightBranch(joinClause, rightInputRef, true);
// A join operator with condition TRUE.
AbstractBinaryJoinOperator joinOperator = new InnerJoinOperator(
new MutableObject<>(ConstantExpression.TRUE), leftInputRef, new MutableObject<>(rightBranch.first));
@@ -362,7 +358,7 @@
filter.getInputs().add(conditionExprOpPair.second);
filter.setSourceLocation(conditionExprOpPair.first.getSourceLocation());
return new Pair<>(filter, rightBranch.second);
- } else {
+ } else if (joinClause.getJoinType() == JoinType.LEFTOUTER) {
// Creates a subplan operator.
SubplanOperator subplanOp = new SubplanOperator();
subplanOp.getInputs().add(leftInputRef);
@@ -498,6 +494,9 @@
context.setVar(joinClause.getRightVariable(), outerUnnestVar);
}
return new Pair<>(currentTopOp, null);
+ } else {
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, joinClause.getSourceLocation(),
+ String.valueOf(joinClause.getJoinType().toString()));
}
}
@@ -980,7 +979,9 @@
QuantifiedPair qp = qe.getQuantifiedList().get(0);
VariableExpr varExpr = qp.getVarExpr();
- List<Expression> operandExprs = ((OperatorExpr) qe.getSatisfiesExpr()).getExprList();
+ OperatorExpr condExpr = (OperatorExpr) qe.getSatisfiesExpr();
+ List<IExpressionAnnotation> condExprHints = condExpr.getHints();
+ List<Expression> operandExprs = condExpr.getExprList();
int varIdx = operandExprs.indexOf(varExpr);
Expression operandExpr = operandExprs.get(1 - varIdx);
@@ -1013,7 +1014,7 @@
throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
itemExpr.getKind());
}
- ILogicalExpression eqExpr = createEqExpr(operandVar, inValue, sourceLoc);
+ ILogicalExpression eqExpr = createEqExpr(operandVar, inValue, condExprHints, sourceLoc);
disjuncts.add(new MutableObject<>(eqExpr));
}
break;
@@ -1024,7 +1025,7 @@
inVarCursor.reset();
while (inVarCursor.next()) {
IAObject inValue = inVarCursor.get();
- ILogicalExpression eqExpr = createEqExpr(operandVar, inValue, sourceLoc);
+ ILogicalExpression eqExpr = createEqExpr(operandVar, inValue, condExprHints, sourceLoc);
disjuncts.add(new MutableObject<>(eqExpr));
}
break;
@@ -1032,25 +1033,25 @@
throw new IllegalStateException(String.valueOf(inExpr.getKind()));
}
- MutableObject<ILogicalExpression> condExpr;
+ MutableObject<ILogicalExpression> disjunctiveExpr;
if (disjuncts.size() == 1) {
- condExpr = disjuncts.get(0);
+ disjunctiveExpr = disjuncts.get(0);
} else {
AbstractFunctionCallExpression orExpr =
createFunctionCallExpressionForBuiltinOperator(OperatorType.OR, sourceLoc);
orExpr.getArguments().addAll(disjuncts);
- condExpr = new MutableObject<>(orExpr);
+ disjunctiveExpr = new MutableObject<>(orExpr);
}
LogicalVariable assignVar = context.newVar();
- AssignOperator assignOp = new AssignOperator(assignVar, condExpr);
+ AssignOperator assignOp = new AssignOperator(assignVar, disjunctiveExpr);
assignOp.getInputs().add(topOp);
assignOp.setSourceLocation(sourceLoc);
return new Pair<>(assignOp, assignVar);
}
- private ILogicalExpression createEqExpr(LogicalVariable lhsVar, IAObject rhsValue, SourceLocation sourceLoc)
- throws CompilationException {
+ private ILogicalExpression createEqExpr(LogicalVariable lhsVar, IAObject rhsValue,
+ List<IExpressionAnnotation> hints, SourceLocation sourceLoc) throws CompilationException {
VariableReferenceExpression lhsExpr = new VariableReferenceExpression(lhsVar);
lhsExpr.setSourceLocation(sourceLoc);
ILogicalExpression rhsExpr = translateConstantValue(rhsValue, sourceLoc);
@@ -1058,6 +1059,11 @@
createFunctionCallExpressionForBuiltinOperator(OperatorType.EQ, sourceLoc);
opExpr.getArguments().add(new MutableObject<>(lhsExpr));
opExpr.getArguments().add(new MutableObject<>(rhsExpr));
+ if (hints != null) {
+ for (IExpressionAnnotation hint : hints) {
+ opExpr.getAnnotations().put(hint, hint);
+ }
+ }
return opExpr;
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java
index 9c6e75e..cb6515b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/TypeTranslator.java
@@ -29,6 +29,8 @@
import org.apache.asterix.common.annotations.IRecordFieldDataGen;
import org.apache.asterix.common.annotations.RecordDataGenAnnotation;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.lang.common.expression.OrderedListTypeDefinition;
import org.apache.asterix.lang.common.expression.RecordTypeDefinition;
import org.apache.asterix.lang.common.expression.RecordTypeDefinition.RecordKind;
@@ -40,6 +42,7 @@
import org.apache.asterix.metadata.entities.BuiltinTypeMap;
import org.apache.asterix.metadata.entities.Datatype;
import org.apache.asterix.metadata.utils.MetadataConstants;
+import org.apache.asterix.metadata.utils.TypeUtil;
import org.apache.asterix.om.types.AOrderedListType;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.AUnionType;
@@ -49,6 +52,7 @@
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.types.TypeSignature;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
public class TypeTranslator {
@@ -69,7 +73,7 @@
firstPass(typeExpr, typeName, typeMap, incompleteFieldTypes, incompleteItemTypes,
incompleteTopLevelTypeReferences, typeDataverse);
secondPass(mdTxnCtx, typeMap, incompleteFieldTypes, incompleteItemTypes, incompleteTopLevelTypeReferences,
- typeDataverse);
+ typeDataverse, typeExpr.getSourceLocation());
for (IAType type : typeMap.values()) {
if (type.getTypeTag().isDerivedType()) {
@@ -86,7 +90,8 @@
throws AlgebricksException {
if (BuiltinTypeMap.getBuiltinType(typeName) != null) {
- throw new AlgebricksException("Cannot redefine builtin type " + typeName + " .");
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR, typeExpr.getSourceLocation(),
+ "Cannot redefine builtin type " + typeName);
}
TypeSignature typeSignature = new TypeSignature(typeDataverse, typeName);
switch (typeExpr.getTypeKind()) {
@@ -132,15 +137,15 @@
private static void secondPass(MetadataTransactionContext mdTxnCtx, Map<TypeSignature, IAType> typeMap,
Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes,
- Map<TypeSignature, List<TypeSignature>> incompleteTopLevelTypeReferences, String typeDataverse)
- throws AlgebricksException {
+ Map<TypeSignature, List<TypeSignature>> incompleteTopLevelTypeReferences, String typeDataverse,
+ SourceLocation sourceLoc) throws AlgebricksException {
// solve remaining top level references
for (TypeSignature typeSignature : incompleteTopLevelTypeReferences.keySet()) {
IAType t;
Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, typeSignature.getNamespace(),
typeSignature.getName());
if (dt == null) {
- throw new AlgebricksException("Could not resolve type " + typeSignature);
+ throw new CompilationException(ErrorCode.UNKNOWN_TYPE, sourceLoc, typeSignature.getName());
} else {
t = dt.getDatatype();
}
@@ -157,7 +162,7 @@
trefName);
}
if (dt == null) {
- throw new AlgebricksException("Could not resolve type " + trefName);
+ throw new CompilationException(ErrorCode.UNKNOWN_TYPE, sourceLoc, trefName);
} else {
t = dt.getDatatype();
}
@@ -184,7 +189,7 @@
dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, typeSignature.getNamespace(),
typeSignature.getName());
if (dt == null) {
- throw new AlgebricksException("Could not resolve type " + typeSignature);
+ throw new CompilationException(ErrorCode.UNKNOWN_TYPE, sourceLoc, typeSignature.getName());
}
t = dt.getDatatype();
} else {
@@ -343,50 +348,35 @@
for (int j = 0; j < n; j++) {
TypeExpression texpr = rtd.getFieldTypes().get(j);
+ IAType type;
switch (texpr.getTypeKind()) {
case TYPEREFERENCE: {
TypeReferenceExpression tre = (TypeReferenceExpression) texpr;
TypeSignature signature = new TypeSignature(
tre.getIdent().first == null ? defaultDataverse : tre.getIdent().first.getValue(),
tre.getIdent().second.getValue());
- IAType tref = solveTypeReference(signature, typeMap);
- if (tref != null) {
- if (!rtd.getOptionableFields().get(j)) { // not nullable
- fldTypes[j] = tref;
- } else { // optional
- fldTypes[j] = AUnionType.createUnknownableType(tref);
- }
- } else {
+ type = solveTypeReference(signature, typeMap);
+ if (type == null) {
addIncompleteFieldTypeReference(recType, j, tre, incompleteFieldTypes);
- if (rtd.getOptionableFields().get(j)) {
- fldTypes[j] = AUnionType.createUnknownableType(null);
- }
}
break;
}
case RECORD: {
RecordTypeDefinition recTypeDef2 = (RecordTypeDefinition) texpr;
- IAType t2 = computeRecordType(null, recTypeDef2, typeMap, incompleteFieldTypes, incompleteItemTypes,
+ type = computeRecordType(null, recTypeDef2, typeMap, incompleteFieldTypes, incompleteItemTypes,
defaultDataverse);
- if (!rtd.getOptionableFields().get(j)) { // not nullable
- fldTypes[j] = t2;
- } else { // nullable
- fldTypes[j] = AUnionType.createUnknownableType(t2);
- }
break;
}
case ORDEREDLIST: {
OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) texpr;
- IAType t2 = computeOrderedListType(null, oltd, typeMap, incompleteItemTypes, incompleteFieldTypes,
+ type = computeOrderedListType(null, oltd, typeMap, incompleteItemTypes, incompleteFieldTypes,
defaultDataverse);
- fldTypes[j] = rtd.getOptionableFields().get(j) ? AUnionType.createUnknownableType(t2) : t2;
break;
}
case UNORDEREDLIST: {
UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) texpr;
- IAType t2 = computeUnorderedListType(null, ultd, typeMap, incompleteItemTypes, incompleteFieldTypes,
+ type = computeUnorderedListType(null, ultd, typeMap, incompleteItemTypes, incompleteFieldTypes,
defaultDataverse);
- fldTypes[j] = rtd.getOptionableFields().get(j) ? AUnionType.createUnknownableType(t2) : t2;
break;
}
default: {
@@ -394,6 +384,9 @@
}
}
+ Boolean nullable = rtd.getNullableFields().get(j);
+ Boolean missable = rtd.getMissableFields().get(j);
+ fldTypes[j] = TypeUtil.createQuantifiedType(type, nullable, missable);
}
return recType;
diff --git a/asterixdb/asterix-app/data/csv/01.csv b/asterixdb/asterix-app/data/csv/01.csv
new file mode 100644
index 0000000..6957e76
--- /dev/null
+++ b/asterixdb/asterix-app/data/csv/01.csv
@@ -0,0 +1,3 @@
+1,,"good","recommend"
+2,,"bad","not recommend"
+3,,"good",
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/csv/02.csv b/asterixdb/asterix-app/data/csv/02.csv
new file mode 100644
index 0000000..630843f
--- /dev/null
+++ b/asterixdb/asterix-app/data/csv/02.csv
@@ -0,0 +1,3 @@
+4,2018,"good","recommend"
+5,2018,,"not recommend"
+6,2018,"good",
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate/avg_mixed/avg_mixed.1.ast b/asterixdb/asterix-app/data/csv/empty.csv
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate/avg_mixed/avg_mixed.1.ast
copy to asterixdb/asterix-app/data/csv/empty.csv
diff --git a/asterixdb/asterix-app/data/csv/empty_lines.csv b/asterixdb/asterix-app/data/csv/empty_lines.csv
new file mode 100644
index 0000000..3f2ff2d
--- /dev/null
+++ b/asterixdb/asterix-app/data/csv/empty_lines.csv
@@ -0,0 +1,5 @@
+
+
+
+
+
diff --git a/asterixdb/asterix-app/data/csv/error1_line_num.csv b/asterixdb/asterix-app/data/csv/error1_line_num.csv
new file mode 100644
index 0000000..34bcee9
--- /dev/null
+++ b/asterixdb/asterix-app/data/csv/error1_line_num.csv
@@ -0,0 +1,3 @@
+1,"good","recommend"
+
+2,"bad" ,"not recommend"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/csv/error2_line_num.csv b/asterixdb/asterix-app/data/csv/error2_line_num.csv
new file mode 100644
index 0000000..0f1286f
--- /dev/null
+++ b/asterixdb/asterix-app/data/csv/error2_line_num.csv
@@ -0,0 +1,5 @@
+1,"good","recommend"
+2,"bad and
+not so good and
+bad" ,"not recommend"
+3,"good","recommend"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/csv/header/h_invalid_values.csv b/asterixdb/asterix-app/data/csv/header/h_invalid_values.csv
new file mode 100644
index 0000000..1ec6140
--- /dev/null
+++ b/asterixdb/asterix-app/data/csv/header/h_invalid_values.csv
@@ -0,0 +1,13 @@
+bigint_t,nullable_bigint_t,double_t,nullable_double_t,str_t,nullable_str_t,boolean_t,nullable_boolean_t
+163a,12,33.4,16.1,"text","text",true,false
+1.22,12,33.4,16.1,"text","text",true,false
+999999999999999999999999999999999,12,33.4,16.1,"text","text",true,false
+1234,12,non-double,16.1,"text","text",true,false
+1234 ,12, 15.1,16.1,"fine","fine",TRUE,false
+1234,12,15.1,16.1,"text","text",non-boolean,false
+1234,12,15.1,16.1,"fine","fine", true,false
+1234,12,15.1,16.1,"text", "quote-not-at-beginning",true,false
+1234,\N,15.1,\N,\N,\N,true,\N
+\N,12,15.1,16.1,"text","text",true,false
+1234,12,\N,16.1,"text","text",true,false
+1234,12,15.1,16.1,"text","text",\N,false
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/csv/header/h_mul_rec.csv b/asterixdb/asterix-app/data/csv/header/h_mul_rec.csv
new file mode 100644
index 0000000..23d0bcd
--- /dev/null
+++ b/asterixdb/asterix-app/data/csv/header/h_mul_rec.csv
@@ -0,0 +1,4 @@
+f1,f2,f3,f4
+1,2,3,"str"
+4,5,6,"rts"
+7,8,9,"srt"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/csv/header/h_mul_rec_with_ln.csv b/asterixdb/asterix-app/data/csv/header/h_mul_rec_with_ln.csv
new file mode 100644
index 0000000..3ea2987
--- /dev/null
+++ b/asterixdb/asterix-app/data/csv/header/h_mul_rec_with_ln.csv
@@ -0,0 +1,4 @@
+f1,f2,f3,f4
+1,2,3,"str"
+4,5,6,"rts"
+7,8,9,"srt"
diff --git a/asterixdb/asterix-app/data/csv/header/h_one_rec.csv b/asterixdb/asterix-app/data/csv/header/h_one_rec.csv
new file mode 100644
index 0000000..236141c
--- /dev/null
+++ b/asterixdb/asterix-app/data/csv/header/h_one_rec.csv
@@ -0,0 +1,2 @@
+f1,f2,f3,f4
+1,2,3,"str"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/csv/header/h_one_rec_with_ln.csv b/asterixdb/asterix-app/data/csv/header/h_one_rec_with_ln.csv
new file mode 100644
index 0000000..83b19b3
--- /dev/null
+++ b/asterixdb/asterix-app/data/csv/header/h_one_rec_with_ln.csv
@@ -0,0 +1,2 @@
+f1,f2,f3,f4
+1,2,3,"str"
diff --git a/asterixdb/asterix-app/data/csv/header/h_only.csv b/asterixdb/asterix-app/data/csv/header/h_only.csv
new file mode 100644
index 0000000..7988898
--- /dev/null
+++ b/asterixdb/asterix-app/data/csv/header/h_only.csv
@@ -0,0 +1 @@
+f1,f2,f3,f4
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/csv/header/h_only_with_ln.csv b/asterixdb/asterix-app/data/csv/header/h_only_with_ln.csv
new file mode 100644
index 0000000..33ddfb1
--- /dev/null
+++ b/asterixdb/asterix-app/data/csv/header/h_only_with_ln.csv
@@ -0,0 +1 @@
+f1,f2,f3,f4
diff --git a/asterixdb/asterix-app/data/csv/no_header/no_h_missing_fields.csv b/asterixdb/asterix-app/data/csv/no_header/no_h_missing_fields.csv
new file mode 100644
index 0000000..8dbe35c
--- /dev/null
+++ b/asterixdb/asterix-app/data/csv/no_header/no_h_missing_fields.csv
@@ -0,0 +1,3 @@
+1,2,3,"str"
+4,5,6
+7,8,9,"srt"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/csv/no_header/no_h_mul_rec.csv b/asterixdb/asterix-app/data/csv/no_header/no_h_mul_rec.csv
new file mode 100644
index 0000000..85abbfb
--- /dev/null
+++ b/asterixdb/asterix-app/data/csv/no_header/no_h_mul_rec.csv
@@ -0,0 +1,3 @@
+1,2,3,"str"
+4,5,6,"rts"
+7,8,9,"srt"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/csv/no_header/no_h_mul_rec_with_ln.csv b/asterixdb/asterix-app/data/csv/no_header/no_h_mul_rec_with_ln.csv
new file mode 100644
index 0000000..e20e795
--- /dev/null
+++ b/asterixdb/asterix-app/data/csv/no_header/no_h_mul_rec_with_ln.csv
@@ -0,0 +1,3 @@
+1,2,3,"str"
+4,5,6,"rts"
+7,8,9,"srt"
diff --git a/asterixdb/asterix-app/data/csv/no_header/no_h_no_closing_q.csv b/asterixdb/asterix-app/data/csv/no_header/no_h_no_closing_q.csv
new file mode 100644
index 0000000..abefcf7
--- /dev/null
+++ b/asterixdb/asterix-app/data/csv/no_header/no_h_no_closing_q.csv
@@ -0,0 +1 @@
+1,2,3,"5
diff --git a/asterixdb/asterix-app/data/csv/no_header/no_h_one_rec.csv b/asterixdb/asterix-app/data/csv/no_header/no_h_one_rec.csv
new file mode 100644
index 0000000..e80e3a2
--- /dev/null
+++ b/asterixdb/asterix-app/data/csv/no_header/no_h_one_rec.csv
@@ -0,0 +1 @@
+1,2,3,"str"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/csv/no_header/no_h_one_rec_with_ln.csv b/asterixdb/asterix-app/data/csv/no_header/no_h_one_rec_with_ln.csv
new file mode 100644
index 0000000..a884ca6
--- /dev/null
+++ b/asterixdb/asterix-app/data/csv/no_header/no_h_one_rec_with_ln.csv
@@ -0,0 +1 @@
+1,2,3,"str"
diff --git a/asterixdb/asterix-app/data/csv/sample_09.csv b/asterixdb/asterix-app/data/csv/sample_09.csv
new file mode 100644
index 0000000..b14219d
--- /dev/null
+++ b/asterixdb/asterix-app/data/csv/sample_09.csv
@@ -0,0 +1,17 @@
+a,b,c,d,e
+0,", boo", 1,2,3
+1,"","",❤,
+2,3,4,\n,
+3,"quoted ""f"" field",,,
+4,4,,,
+5,"{""vehicle"": ""car"", ""location"": [2.0, 0.1]}",,,
+6,2,3,,
+7,8,9,,
+8,2,3,,
+9,8,9,,
+10,"field
+""f""
+with multiple lines",,,
+11,4,,,
+12,5,ʤ,,
+John,Green,111 downtown st.,"city, state",99999
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/csv/sample_10.csv b/asterixdb/asterix-app/data/csv/sample_10.csv
new file mode 100644
index 0000000..3beee08
--- /dev/null
+++ b/asterixdb/asterix-app/data/csv/sample_10.csv
@@ -0,0 +1,39 @@
+1,"?/ Text ending with a backslash / \",2000-09-03 07:12:22
+2,non quoted text!yes......,2003-08-09 22:34:19
+3,Text with more sentences. Another sentence.,2003-09-12 05:29:12
+4,"Quoted text.. yes.",2003-09-13 17:21:49
+5,Another text,2003-01-21 23:31:41
+6,Text with' quotes.,2003-09-14 20:15:50
+7,Text with quote's,2003-09-14 18:34:03
+8,"Text with quotes '",2003-01-28 20:32:13
+9,"Text with quotes """,2003-01-18 11:44:15
+10,Text with question marks!?!?,2003-09-18 06:25:56
+11,""" Text that starts with quotes",2003-09-12 00:31:24
+12,"Text with \"" backslash and quotes",2003-09-13 20:30:06
+13,"Text with \"" backslash and quotes\""",2003-09-14 16:20:36
+14,"Text that has comma ,",2003-09-12 08:21:18
+15,"Text that has "","" quoted comma",2003-09-12 08:21:18
+16,",Text that has ",2003-09-12 08:21:18
+17,","",Text that has ",2003-09-12 08:21:18
+18,"Text with commas,inside it., yes",2003-09-13 23:42:14
+19,"Text that has \n inside ",2003-09-12 08:21:18
+20,"Text that has \\\n inside ",2003-09-12 08:21:18
+21,text with :),2003-09-05 19:15:34
+22,"Text that has \\\"" inside \\",2003-09-12 08:21:18
+23,"Text that has \\\"" inside \\""",2003-09-12 08:21:18
+24,"""text that spans multiple
+Lines and more
+Lines ane more and more
+Lines ...
+And yet more lines
+And more""",2011-09-19 01:09:09
+25,"Text ""
+more lines",2011-09-19 01:09:09
+26,"""
+",2011-09-19 01:09:09
+27,"Text",""
+28,"Text","2011-09-19 01:09:09"
+29,"Text\.","2011-09-19 01:09:09"
+30,Text\.,"2011-09-19 01:09:09"
+31,"\.Text","2011-09-19 01:09:09"
+32,\.Text,"2011-09-19 01:09:09"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/csv/sample_11.csv b/asterixdb/asterix-app/data/csv/sample_11.csv
new file mode 100644
index 0000000..b9a9571
--- /dev/null
+++ b/asterixdb/asterix-app/data/csv/sample_11.csv
@@ -0,0 +1,4 @@
+1,","", b", 3,4,5
+","", b",4, 3,4,5
+,,,,
+"dd",,,,
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/csv/sample_12.csv b/asterixdb/asterix-app/data/csv/sample_12.csv
new file mode 100644
index 0000000..0c9baf5
--- /dev/null
+++ b/asterixdb/asterix-app/data/csv/sample_12.csv
@@ -0,0 +1,16 @@
+f1,f2,f3
+1,true,"text"
+2,false,"text"
+3,true,"text"
+4,true,""
+5,false,
+6,true,"text""
+more lines"
+7,false,"""
+"
+8,true,""
+9,false,"text"""
+10,false,text\.
+11,true,"text\."
+,false,\.text
+13,true,"\.text"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/csv/sample_13.csv b/asterixdb/asterix-app/data/csv/sample_13.csv
new file mode 100644
index 0000000..9f53f56
--- /dev/null
+++ b/asterixdb/asterix-app/data/csv/sample_13.csv
@@ -0,0 +1,11 @@
+
+
+f1,f2,f3,f4
+
+1,,"good","recommend"
+
+2,,"bad","not recommend"
+3,,"good",
+
+
+
diff --git a/asterixdb/asterix-app/data/json/malformed-json-2.json b/asterixdb/asterix-app/data/json/malformed-json-2.json
new file mode 100644
index 0000000..88ef9f2
--- /dev/null
+++ b/asterixdb/asterix-app/data/json/malformed-json-2.json
@@ -0,0 +1,7 @@
+{"field": 1,
+ "field2": {
+ "nested1": 9,
+ "array_f": [1, 2,],
+ "nested2": 10
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/json/malformed-json-no-closing.json b/asterixdb/asterix-app/data/json/malformed-json-no-closing.json
new file mode 100644
index 0000000..83f3087
--- /dev/null
+++ b/asterixdb/asterix-app/data/json/malformed-json-no-closing.json
@@ -0,0 +1,2 @@
+{ "field1": 1, "field2": "text"
+
diff --git a/asterixdb/asterix-app/data/json/multi-lines-with-arrays/5-records.json b/asterixdb/asterix-app/data/json/multi-lines-with-arrays/5-records.json
new file mode 100644
index 0000000..fd8526b
--- /dev/null
+++ b/asterixdb/asterix-app/data/json/multi-lines-with-arrays/5-records.json
@@ -0,0 +1,19 @@
+{"id": 1, "year": null, "quarter": null, "review": "good", "array": [1, 2, 3]}
+
+
+{"id": 2, "year": null, "quarter": null, "review": "good", "array": [1, [1, 2], [1]]}
+{"id": 3, "year": 2018, "quarter": null, "review": "good", "array": [1,
+2, 3]}
+
+{"id": 4,
+ "year": 2018,
+ "quarter": null,
+ "review": "bad",
+ "array": [
+ 1,
+ 2,
+ 3
+ ]
+}
+{"id": 5,
+ "year": 2018, "quarter": 1, "review": "good", "array": [1, 2, 3]}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/json/multi-lines-with-nested-objects/5-records.json b/asterixdb/asterix-app/data/json/multi-lines-with-nested-objects/5-records.json
new file mode 100644
index 0000000..b89e81e
--- /dev/null
+++ b/asterixdb/asterix-app/data/json/multi-lines-with-nested-objects/5-records.json
@@ -0,0 +1,32 @@
+{"id": 1, "year": null, "quarter": null, "review": "good", "array": [1, 2, 3], "nested": { "id" : 1}}
+
+
+{"id": 2, "year": null, "quarter": null, "review": "good", "array": [1, [1, 2], [1]], "nested": { "id" : 1}, "nested2": [ {"id":1 }]}
+{"id":3,"year":2018,"quarter":null,"review":"good","array":[1,2,3,{"nested":{"array":[{"nested":{"array":[1,2]}}]}}]}
+
+{
+ "id": 4,
+ "year": 2018,
+ "quarter": null,
+ "review": "bad",
+ "array": [
+ 1,
+ 2,
+ 3,
+ {
+ "nested1": {
+ "id": 1,
+ "nested2": {
+ "id": 2,
+ "nested3": [
+ {
+ "nested4": null
+ }
+ ]
+ }
+ }
+ }
+ ]
+}
+{"id": 5,
+ "year": 2018, "quarter": 1, "review": "good", "array": [1, 2, 3, {"nested": { "array": [1, 2]}}]}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/json/multi-lines/20-records.json b/asterixdb/asterix-app/data/json/multi-lines/20-records.json
new file mode 100644
index 0000000..0591790
--- /dev/null
+++ b/asterixdb/asterix-app/data/json/multi-lines/20-records.json
@@ -0,0 +1,34 @@
+{"id": 1, "year": null, "quarter": null, "review": "good"}
+
+
+{"id": 2, "year": null, "quarter": null, "review": "good"}
+{"id": 3, "year": 2018, "quarter": null, "review": "good"}
+
+{"id": 4,
+ "year": 2018,
+ "quarter": null,
+ "review": "bad"
+}
+{"id": 5,
+ "year": 2018, "quarter": 1, "review": "good"}
+{"id": 6, "year": 2018, "quarter": 1, "review": "bad"
+}
+{"id": 7, "year": 2018, "quarter": 2, "review": "good"}
+{"id": 8, "year": 2018, "quarter": 2, "review": "bad"}
+{"id": 9, "year": 2019, "quarter": null,
+
+ "review": "good"}
+{"id": 10, "year": 2019,
+
+ "quarter": null,
+ "review": "bad"}
+{"id": 11, "year": 2019, "quarter": 1, "review": "good"}
+{"id": 12, "year": 2019, "quarter": 1, "review": "bad"}
+{"id": 13, "year": 2019, "quarter": 2, "review": "good"}
+{"id": 14, "year": 2019, "quarter": 2, "review": "bad"}
+{"id": 15, "year": 2019, "quarter": null, "review": "good"}
+{"id": 16, "year": 2019, "quarter": null, "review": "bad"}
+{"id": 17, "year": 2019, "quarter": 1, "review": "good"}
+{"id": 18, "year": 2019, "quarter": 1, "review": "bad"}
+{"id": 19, "year": 2019, "quarter": 2, "review": "good"}
+{"id": 20, "year": 2019, "quarter": 2, "review": "bad"}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/json/single-line/20-records.json b/asterixdb/asterix-app/data/json/single-line/20-records.json
new file mode 100644
index 0000000..01176da
--- /dev/null
+++ b/asterixdb/asterix-app/data/json/single-line/20-records.json
@@ -0,0 +1,20 @@
+{"id": 1, "year": null, "quarter": null, "review": "good"}
+{"id": 2, "year": null, "quarter": null, "review": "good"}
+{"id": 3, "year": 2018, "quarter": null, "review": "good"}
+{"id": 4, "year": 2018, "quarter": null, "review": "bad"}
+{"id": 5, "year": 2018, "quarter": 1, "review": "good"}
+{"id": 6, "year": 2018, "quarter": 1, "review": "bad"}
+{"id": 7, "year": 2018, "quarter": 2, "review": "good"}
+{"id": 8, "year": 2018, "quarter": 2, "review": "bad"}
+{"id": 9, "year": 2019, "quarter": null, "review": "good"}
+{"id": 10, "year": 2019, "quarter": null, "review": "bad"}
+{"id": 11, "year": 2019, "quarter": 1, "review": "good"}
+{"id": 12, "year": 2019, "quarter": 1, "review": "bad"}
+{"id": 13, "year": 2019, "quarter": 2, "review": "good"}
+{"id": 14, "year": 2019, "quarter": 2, "review": "bad"}
+{"id": 15, "year": 2019, "quarter": null, "review": "good"}
+{"id": 16, "year": 2019, "quarter": null, "review": "bad"}
+{"id": 17, "year": 2019, "quarter": 1, "review": "good"}
+{"id": 18, "year": 2019, "quarter": 1, "review": "bad"}
+{"id": 19, "year": 2019, "quarter": 2, "review": "good"}
+{"id": 20, "year": 2019, "quarter": 2, "review": "bad"}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/json/single-line/array_of_objects.json b/asterixdb/asterix-app/data/json/single-line/array_of_objects.json
new file mode 100644
index 0000000..5ea91f9
--- /dev/null
+++ b/asterixdb/asterix-app/data/json/single-line/array_of_objects.json
@@ -0,0 +1 @@
+[{"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}, {"mutated": 0, "filename": "file_0.json", "folder": "", "null": null, "missing": 0}]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/jsonl/malformed-jsonl-1.json b/asterixdb/asterix-app/data/jsonl/malformed-jsonl-1.json
new file mode 100644
index 0000000..7d15a9c
--- /dev/null
+++ b/asterixdb/asterix-app/data/jsonl/malformed-jsonl-1.json
@@ -0,0 +1,4 @@
+{"field": 1, "field2": true}
+{"field": 2, "field2": false}
+{"field": 3, "field2": truee}
+{"field": 4, "field2": true}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/jsonl/malformed-jsonl-2.json b/asterixdb/asterix-app/data/jsonl/malformed-jsonl-2.json
new file mode 100644
index 0000000..ebd3538
--- /dev/null
+++ b/asterixdb/asterix-app/data/jsonl/malformed-jsonl-2.json
@@ -0,0 +1,14 @@
+{"field": 1,
+ "field2": {
+ "nested1": 8,
+ "array_f": [1, 2],
+ "nested2": 9
+ }
+}
+{"field": 2,
+ "field2": {
+ "nested1": 88,
+ "array_f": [11, 22, ],
+ "nested2": 99
+ }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/jsonl/multi-lines-with-arrays/5-records.json b/asterixdb/asterix-app/data/jsonl/multi-lines-with-arrays/5-records.json
new file mode 100644
index 0000000..5bd8cb6
--- /dev/null
+++ b/asterixdb/asterix-app/data/jsonl/multi-lines-with-arrays/5-records.json
@@ -0,0 +1,20 @@
+[]
+{"id": 21, "year": null, "quarter": null, "review": "good", "array": [1, 2, 3]}
+
+
+{"id": 22, "year": null, "quarter": null, "review": "good", "array": [1, [1, 2], [1]]}
+{"id": 23, "year": 2018, "quarter": null, "review": "good", "array": [1,
+2, 3]}
+
+{"id": 24,
+ "year": 2018,
+ "quarter": null,
+ "review": "bad",
+ "array": [
+ 1,
+ 2,
+ 3
+ ]
+}
+{"id": 25,
+ "year": 2018, "quarter": 1, "review": "good", "array": [1, 2, 3]}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/jsonl/multi-lines-with-nested-objects/5-records.json b/asterixdb/asterix-app/data/jsonl/multi-lines-with-nested-objects/5-records.json
new file mode 100644
index 0000000..063e30a
--- /dev/null
+++ b/asterixdb/asterix-app/data/jsonl/multi-lines-with-nested-objects/5-records.json
@@ -0,0 +1,33 @@
+{"id": 26, "year": null, "quarter": null, "review": "good", "array": [1, 2, 3], "nested": { "id" : 1}}
+
+[]
+[ ]
+[{"id": 27, "year": null, "quarter": null, "review": "good", "array": [1, [1, 2], [1]], "nested": { "id" : 1}, "nested2": [ {"id":1 }]},
+{"id":28,"year":2018,"quarter":null,"review":"good","array":[1,2,3,{"nested":{"array":[{"nested":{"array":[1,2]}}]}}]}]
+
+{
+ "id": 29,
+ "year": 2018,
+ "quarter": null,
+ "review": "bad",
+ "array": [
+ 1,
+ 2,
+ 3,
+ {
+ "nested1": {
+ "id": 1,
+ "nested2": {
+ "id": 2,
+ "nested3": [
+ {
+ "nested4": null
+ }
+ ]
+ }
+ }
+ }
+ ]
+}
+{"id": 30,
+ "year": 2018, "quarter": 1, "review": "good", "array": [1, 2, 3, {"nested": { "array": [1, 2]}}]}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/jsonl/multi-lines/20-records.json b/asterixdb/asterix-app/data/jsonl/multi-lines/20-records.json
new file mode 100644
index 0000000..69aa775
--- /dev/null
+++ b/asterixdb/asterix-app/data/jsonl/multi-lines/20-records.json
@@ -0,0 +1,33 @@
+[{"id": 1, "year": null, "quarter": null, "review": "good"},
+
+
+{"id": 2, "year": null, "quarter": null, "review": "good"} ,
+{"id": 3, "year": 2018, "quarter": null, "review": "good"}]
+
+{"id": 4,
+ "year": 2018,
+ "quarter": null,
+ "review": "bad"
+}
+{"id": 5,
+ "year": 2018, "quarter": 1, "review": "good"}
+{"id": 6, "year": 2018, "quarter": 1, "review": "bad"
+}
+{"id": 7, "year": 2018, "quarter": 2, "review": "good"}
+{"id": 8, "year": 2018, "quarter": 2, "review": "bad"}
+[{"id": 9, "year": 2019, "quarter": null,
+
+ "review": "good"},
+{"id": 10, "year": 2019,
+
+ "quarter": null,
+ "review": "bad"} ,
+{"id": 11, "year": 2019, "quarter": 1, "review": "good"}]
+[{"id": 12, "year": 2019, "quarter": 1, "review": "bad"}]
+{"id": 13, "year": 2019, "quarter": 2, "review": "good"}
+{"id": 14, "year": 2019, "quarter": 2, "review": "bad"}
+{"id": 15, "year": 2019, "quarter": null, "review": "good"}
+{"id": 16, "year": 2019, "quarter": null, "review": "bad"}
+{"id": 17, "year": 2019, "quarter": 1, "review": "good"}
+[{"id": 18, "year": 2019, "quarter": 1, "review": "bad"}, {"id": 19, "year": 2019, "quarter": 2, "review": "good"},
+{"id": 20, "year": 2019, "quarter": 2, "review": "bad"}]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/jsonl/single-line/20-records.json b/asterixdb/asterix-app/data/jsonl/single-line/20-records.json
new file mode 100644
index 0000000..bf86095
--- /dev/null
+++ b/asterixdb/asterix-app/data/jsonl/single-line/20-records.json
@@ -0,0 +1,20 @@
+[{"id": 31, "year": null, "quarter": null, "review": "good"},
+{"id": 32, "year": null, "quarter": null, "review": "good"},
+{"id": 33, "year": 2018, "quarter": null, "review": "good"},
+{"id": 34, "year": 2018, "quarter": null, "review": "bad"},
+{"id": 35, "year": 2018, "quarter": 1, "review": "good"},
+{"id": 36, "year": 2018, "quarter": 1, "review": "bad"},
+{"id": 37, "year": 2018, "quarter": 2, "review": "good"},
+{"id": 38, "year": 2018, "quarter": 2, "review": "bad"},
+{"id": 39, "year": 2019, "quarter": null, "review": "good"},
+{"id": 40, "year": 2019, "quarter": null, "review": "bad"},
+{"id": 41, "year": 2019, "quarter": 1, "review": "good"},
+{"id": 42, "year": 2019, "quarter": 1, "review": "bad"},
+{"id": 43, "year": 2019, "quarter": 2, "review": "good"},
+{"id": 44, "year": 2019, "quarter": 2, "review": "bad"},
+{"id": 45, "year": 2019, "quarter": null, "review": "good"},
+{"id": 46, "year": 2019, "quarter": null, "review": "bad"},
+{"id": 47, "year": 2019, "quarter": 1, "review": "good"},
+{"id": 48, "year": 2019, "quarter": 1, "review": "bad"},
+{"id": 49, "year": 2019, "quarter": 2, "review": "good"},
+{"id": 50, "year": 2019, "quarter": 2, "review": "bad"}]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/tsv/01.tsv b/asterixdb/asterix-app/data/tsv/01.tsv
new file mode 100644
index 0000000..98876c7
--- /dev/null
+++ b/asterixdb/asterix-app/data/tsv/01.tsv
@@ -0,0 +1,3 @@
+1 "good" "recommend"
+2 "bad" "not recommend"
+3 "good" "recommend"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/tsv/02.tsv b/asterixdb/asterix-app/data/tsv/02.tsv
new file mode 100644
index 0000000..c01ce7c
--- /dev/null
+++ b/asterixdb/asterix-app/data/tsv/02.tsv
@@ -0,0 +1,3 @@
+4 2018 "good" "recommend"
+5 2018 "not recommend"
+6 2018 "good" "recommend"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate/avg_mixed/avg_mixed.1.ast b/asterixdb/asterix-app/data/tsv/empty.tsv
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate/avg_mixed/avg_mixed.1.ast
copy to asterixdb/asterix-app/data/tsv/empty.tsv
diff --git a/asterixdb/asterix-app/data/tsv/empty_lines.tsv b/asterixdb/asterix-app/data/tsv/empty_lines.tsv
new file mode 100644
index 0000000..3f2ff2d
--- /dev/null
+++ b/asterixdb/asterix-app/data/tsv/empty_lines.tsv
@@ -0,0 +1,5 @@
+
+
+
+
+
diff --git a/asterixdb/asterix-app/data/tsv/header/h_mul_rec.tsv b/asterixdb/asterix-app/data/tsv/header/h_mul_rec.tsv
new file mode 100644
index 0000000..ba73bb2
--- /dev/null
+++ b/asterixdb/asterix-app/data/tsv/header/h_mul_rec.tsv
@@ -0,0 +1,4 @@
+f1 f2 f3 f4
+1 2 3 "str"
+4 5 6 "rts"
+7 8 9 "srt"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/tsv/header/h_mul_rec_with_ln.tsv b/asterixdb/asterix-app/data/tsv/header/h_mul_rec_with_ln.tsv
new file mode 100644
index 0000000..eaf1fab
--- /dev/null
+++ b/asterixdb/asterix-app/data/tsv/header/h_mul_rec_with_ln.tsv
@@ -0,0 +1,4 @@
+f1 f2 f3 f4
+1 2 3 "str"
+4 5 6 "rts"
+7 8 9 "srt"
diff --git a/asterixdb/asterix-app/data/tsv/header/h_one_rec.tsv b/asterixdb/asterix-app/data/tsv/header/h_one_rec.tsv
new file mode 100644
index 0000000..167b56b
--- /dev/null
+++ b/asterixdb/asterix-app/data/tsv/header/h_one_rec.tsv
@@ -0,0 +1,2 @@
+f1 f2 f3 f4
+1 2 3 "str"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/tsv/header/h_one_rec_with_ln.tsv b/asterixdb/asterix-app/data/tsv/header/h_one_rec_with_ln.tsv
new file mode 100644
index 0000000..95d2d55
--- /dev/null
+++ b/asterixdb/asterix-app/data/tsv/header/h_one_rec_with_ln.tsv
@@ -0,0 +1,2 @@
+f1 f2 f3 f4
+1 2 3 "str"
diff --git a/asterixdb/asterix-app/data/tsv/header/h_only.tsv b/asterixdb/asterix-app/data/tsv/header/h_only.tsv
new file mode 100644
index 0000000..b72029a
--- /dev/null
+++ b/asterixdb/asterix-app/data/tsv/header/h_only.tsv
@@ -0,0 +1 @@
+f1 f2 f3 f4
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/tsv/header/h_only_with_ln.tsv b/asterixdb/asterix-app/data/tsv/header/h_only_with_ln.tsv
new file mode 100644
index 0000000..8180e15
--- /dev/null
+++ b/asterixdb/asterix-app/data/tsv/header/h_only_with_ln.tsv
@@ -0,0 +1 @@
+f1 f2 f3 f4
diff --git a/asterixdb/asterix-app/data/tsv/no_header/no_h_missing_fields.tsv b/asterixdb/asterix-app/data/tsv/no_header/no_h_missing_fields.tsv
new file mode 100644
index 0000000..75002ac
--- /dev/null
+++ b/asterixdb/asterix-app/data/tsv/no_header/no_h_missing_fields.tsv
@@ -0,0 +1,3 @@
+1 2 3 "str"
+4 5 6
+7 8 9 "srt"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/tsv/no_header/no_h_mul_rec.tsv b/asterixdb/asterix-app/data/tsv/no_header/no_h_mul_rec.tsv
new file mode 100644
index 0000000..517974a
--- /dev/null
+++ b/asterixdb/asterix-app/data/tsv/no_header/no_h_mul_rec.tsv
@@ -0,0 +1,3 @@
+1 2 3 "str"
+4 5 6 "rts"
+7 8 9 "srt"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/tsv/no_header/no_h_mul_rec_with_ln.tsv b/asterixdb/asterix-app/data/tsv/no_header/no_h_mul_rec_with_ln.tsv
new file mode 100644
index 0000000..869fefb
--- /dev/null
+++ b/asterixdb/asterix-app/data/tsv/no_header/no_h_mul_rec_with_ln.tsv
@@ -0,0 +1,3 @@
+1 2 3 "str"
+4 5 6 "rts"
+7 8 9 "srt"
diff --git a/asterixdb/asterix-app/data/tsv/no_header/no_h_one_rec.tsv b/asterixdb/asterix-app/data/tsv/no_header/no_h_one_rec.tsv
new file mode 100644
index 0000000..d076cbe
--- /dev/null
+++ b/asterixdb/asterix-app/data/tsv/no_header/no_h_one_rec.tsv
@@ -0,0 +1 @@
+1 2 3 "str"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/data/tsv/no_header/no_h_one_rec_with_ln.tsv b/asterixdb/asterix-app/data/tsv/no_header/no_h_one_rec_with_ln.tsv
new file mode 100644
index 0000000..53d0de6
--- /dev/null
+++ b/asterixdb/asterix-app/data/tsv/no_header/no_h_one_rec_with_ln.tsv
@@ -0,0 +1 @@
+1 2 3 "str"
diff --git a/asterixdb/asterix-app/data/tsv/sample_01.tsv b/asterixdb/asterix-app/data/tsv/sample_01.tsv
new file mode 100644
index 0000000..aab289a
--- /dev/null
+++ b/asterixdb/asterix-app/data/tsv/sample_01.tsv
@@ -0,0 +1,28 @@
+11 55 text field wih , charrrrrrrrrrr true 90 0.666666667
+12 55 text field with " charrrrrrrrrr false 90 0.666666667
+14 55 text field with ' charrrrrrrrrr false 90 0.666666667
+15 55 text field with \ charrrrrrrrrr false 90 0.666666667
+16 55 text field wih \, char true 90 0.666666667
+17 55 text field with \" charrrrrrrrr false 90 0.666666667
+18 55 text field with \' charrrrrrrrr false 90 0.666666667
+19 55 text field with \\ charrrrrrrrr false 90 0.666666667
+20 55 text field ending with charr , false 90 0.666666667
+21 55 text field ending with charr " false 90 0.666666667
+22 55 text field ending with charr ' false 90 0.666666667
+23 55 text field ending with charr \ false 90 0.666666667
+24 55 text field ending with charr \, false 90 0.666666667
+25 55 text field ending with charr \" false 90 0.666666667
+26 55 text field ending with charr \' false 90 0.666666667
+27 55 text field ending with charr \\ false 90 0.666666667
+28 55 ,text field starting with charr false 90 0.666666667
+29 55 "text field starting with charr false 90 0.666666667
+30 55 'text field starting with charr false 90 0.666666667
+31 55 \text field starting with charr false 90 0.666666667
+32 55 \,text field starting with char false 90 0.666666667
+33 55 \"text field starting with char false 90 0.666666667
+34 55 \'text field starting with char false 90 0.666666667
+35 55 \\text field starting with char false 90 0.666666667
+36 55 "text field inside with char" false 90 0.666666667
+37 55 text field with charrrrrrrrr false 90 0.666666667
+38 55 text field with "" charrrrrrrrr false 90 0.666666667
+39 55 text field "with" charrrrrrrrrr false 90 0.666666667
\ No newline at end of file
diff --git a/asterixdb/asterix-app/pom.xml b/asterixdb/asterix-app/pom.xml
index b82898e..4b1fecf 100644
--- a/asterixdb/asterix-app/pom.xml
+++ b/asterixdb/asterix-app/pom.xml
@@ -415,8 +415,7 @@
</dependency>
<dependency>
<groupId>org.mockito</groupId>
- <artifactId>mockito-all</artifactId>
- <version>1.10.19</version>
+ <artifactId>mockito-core</artifactId>
<scope>test</scope>
</dependency>
<dependency>
@@ -726,5 +725,34 @@
<groupId>io.netty</groupId>
<artifactId>netty-common</artifactId>
</dependency>
+ <!-- AWS -->
+ <dependency>
+ <groupId>software.amazon.awssdk</groupId>
+ <artifactId>sdk-core</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>software.amazon.awssdk</groupId>
+ <artifactId>s3</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>software.amazon.awssdk</groupId>
+ <artifactId>regions</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>software.amazon.awssdk</groupId>
+ <artifactId>auth</artifactId>
+ </dependency>
+ <!-- Mock for AWS S3 -->
+ <dependency>
+ <groupId>io.findify</groupId>
+ <artifactId>s3mock_2.12</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <!-- Needed for the s3 mock -->
+ <dependency>
+ <groupId>com.typesafe.akka</groupId>
+ <artifactId>akka-http-core_2.12</artifactId>
+ <scope>test</scope>
+ </dependency>
</dependencies>
</project>
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index d573b52..c863080 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -134,11 +134,12 @@
CompilerProperties.COMPILER_SORTMEMORY_KEY, CompilerProperties.COMPILER_WINDOWMEMORY_KEY,
CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY, CompilerProperties.COMPILER_PARALLELISM_KEY,
CompilerProperties.COMPILER_SORT_PARALLEL_KEY, CompilerProperties.COMPILER_SORT_SAMPLES_KEY,
- FunctionUtil.IMPORT_PRIVATE_FUNCTIONS, FuzzyUtils.SIM_FUNCTION_PROP_NAME,
- FuzzyUtils.SIM_THRESHOLD_PROP_NAME, StartFeedStatement.WAIT_FOR_COMPLETION,
- FeedActivityDetails.FEED_POLICY_NAME, FeedActivityDetails.COLLECT_LOCATIONS,
- SqlppQueryRewriter.INLINE_WITH_OPTION, SqlppExpressionToPlanTranslator.REWRITE_IN_AS_OR_OPTION,
- "hash_merge", "output-record-type", AbstractIntroduceAccessMethodRule.NO_INDEX_ONLY_PLAN_OPTION,
+ CompilerProperties.COMPILER_EXTERNALSCANMEMORY_KEY, FunctionUtil.IMPORT_PRIVATE_FUNCTIONS,
+ FuzzyUtils.SIM_FUNCTION_PROP_NAME, FuzzyUtils.SIM_THRESHOLD_PROP_NAME,
+ StartFeedStatement.WAIT_FOR_COMPLETION, FeedActivityDetails.FEED_POLICY_NAME,
+ FeedActivityDetails.COLLECT_LOCATIONS, SqlppQueryRewriter.INLINE_WITH_OPTION,
+ SqlppExpressionToPlanTranslator.REWRITE_IN_AS_OR_OPTION, "hash_merge", "output-record-type",
+ AbstractIntroduceAccessMethodRule.NO_INDEX_ONLY_PLAN_OPTION,
DisjunctivePredicateToJoinRule.REWRITE_OR_AS_JOIN_OPTION);
private final IRewriterFactory rewriterFactory;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
index aff7441..47e897d 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
@@ -27,6 +27,7 @@
import java.io.InputStreamReader;
import java.io.OutputStream;
import java.io.PrintWriter;
+import java.nio.charset.StandardCharsets;
import java.util.List;
import java.util.concurrent.ConcurrentMap;
@@ -154,7 +155,7 @@
long startTime = System.currentTimeMillis();
final IRequestParameters requestParameters = new RequestParameters(requestReference, query, resultSet,
new ResultProperties(IStatementExecutor.ResultDelivery.IMMEDIATE), new IStatementExecutor.Stats(),
- null, null, null, null, true);
+ new IStatementExecutor.StatementProperties(), null, null, null, null, true);
translator.compileAndExecute(hcc, requestParameters);
long endTime = System.currentTimeMillis();
duration = (endTime - startTime) / 1000.00;
@@ -204,7 +205,7 @@
}
String type = HttpUtil.mime(StaticResourceServlet.extension(resourcePath));
HttpUtil.setContentType(response, "".equals(type) ? HttpUtil.ContentType.TEXT_PLAIN : type,
- HttpUtil.Encoding.UTF8);
+ StandardCharsets.UTF_8);
writeOutput(response, is, resourcePath);
} catch (IOException e) {
LOGGER.log(Level.WARN, "Failure handling request", e);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
index 6953d1f..1b2b00d 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
@@ -69,8 +69,9 @@
@Override
protected void executeStatement(IRequestReference requestReference, String statementsText,
- SessionOutput sessionOutput, ResultProperties resultProperties, IStatementExecutor.Stats stats,
- QueryServiceRequestParameters param, RequestExecutionState execution,
+ SessionOutput sessionOutput, ResultProperties resultProperties,
+ IStatementExecutor.StatementProperties statementProperties, IStatementExecutor.Stats stats,
+ QueryServiceRequestParameters param, RequestExecutionState executionState,
Map<String, String> optionalParameters, Map<String, byte[]> statementParameters,
ResponsePrinter responsePrinter, List<Warning> warnings) throws Exception {
// Running on NC -> send 'execute' message to CC
@@ -89,7 +90,7 @@
resultProperties.getNcToCcResultProperties(), param.getClientContextID(), handleUrl,
optionalParameters, statementParameters, param.isMultiStatement(), param.getProfileType(),
stmtCategoryRestrictionMask, requestReference);
- execution.start();
+ executionState.start();
ncMb.sendMessageToPrimaryCC(requestMsg);
try {
responseMsg = (ExecuteStatementResponseMessage) responseFuture.get(timeout, TimeUnit.MILLISECONDS);
@@ -103,11 +104,12 @@
cancelQuery(ncMb, ncCtx.getNodeId(), requestReference.getUuid(), param.getClientContextID(), hde, true);
throw hde;
}
- execution.end();
+ executionState.end();
} finally {
ncMb.deregisterMessageFuture(responseFuture.getFutureId());
}
+ updatePropertiesFromCC(statementProperties, responseMsg);
Throwable err = responseMsg.getError();
if (err != null) {
if (err instanceof Error) {
@@ -118,6 +120,8 @@
throw new Exception(err.toString(), err);
}
}
+ // if the was no error, we can set the result status to success
+ executionState.setStatus(ResultStatus.SUCCESS, HttpResponseStatus.OK);
updateStatsFromCC(stats, responseMsg);
if (hasResult(responseMsg)) {
responsePrinter.addResultPrinter(
@@ -151,14 +155,14 @@
}
@Override
- protected void handleExecuteStatementException(Throwable t, RequestExecutionState state,
+ protected void handleExecuteStatementException(Throwable t, RequestExecutionState executionState,
QueryServiceRequestParameters param) {
if (t instanceof TimeoutException // TODO(mblow): I don't think t can ever been an instance of TimeoutException
|| ExceptionUtils.matchingCause(t, candidate -> candidate instanceof IPCException)) {
GlobalConfig.ASTERIX_LOGGER.log(Level.WARN, t.toString(), t);
- state.setStatus(ResultStatus.FAILED, HttpResponseStatus.SERVICE_UNAVAILABLE);
+ executionState.setStatus(ResultStatus.FAILED, HttpResponseStatus.SERVICE_UNAVAILABLE);
} else {
- super.handleExecuteStatementException(t, state, param);
+ super.handleExecuteStatementException(t, executionState, param);
}
}
@@ -177,4 +181,13 @@
stats.setProcessedObjects(responseStats.getProcessedObjects());
stats.updateTotalWarningsCount(responseStats.getTotalWarningsCount());
}
+
+ private static void updatePropertiesFromCC(IStatementExecutor.StatementProperties statementProperties,
+ ExecuteStatementResponseMessage responseMsg) {
+ IStatementExecutor.StatementProperties responseStmtProps = responseMsg.getStatementProperties();
+ if (responseStmtProps != null) {
+ statementProperties.setKind(responseStmtProps.getKind());
+ statementProperties.setName(responseStmtProps.getName());
+ }
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
index 5acdf3b..f705b89 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
@@ -112,8 +112,7 @@
ResultUtil.printResults(appCtx, resultReader, sessionOutput, new Stats(), null);
}
} catch (HyracksDataException e) {
- final int errorCode = e.getErrorCode();
- if (ErrorCode.NO_RESULT_SET == errorCode) {
+ if (e.matches(ErrorCode.NO_RESULT_SET)) {
LOGGER.log(Level.INFO, "No results for: \"" + strHandle + "\"");
response.setStatus(HttpResponseStatus.NOT_FOUND);
return;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceRequestParameters.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceRequestParameters.java
index 6e70d27..06f8325 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceRequestParameters.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceRequestParameters.java
@@ -51,6 +51,8 @@
import com.fasterxml.jackson.databind.node.ObjectNode;
import com.google.common.collect.ImmutableMap;
+import io.netty.handler.codec.http.HttpMethod;
+
public class QueryServiceRequestParameters {
public enum Parameter {
@@ -117,6 +119,7 @@
private ResultDelivery mode = ResultDelivery.IMMEDIATE;
private PlanFormat planFormat = PlanFormat.JSON;
private ProfileType profileType = ProfileType.COUNTS;
+ private Map<String, String> optionalParams = null;
private Map<String, JsonNode> statementParams = null;
private boolean pretty = false;
private boolean expressionTree = false;
@@ -220,6 +223,14 @@
this.planFormat = planFormat;
}
+ public Map<String, String> getOptionalParams() {
+ return optionalParams;
+ }
+
+ public void setOptionalParams(Map<String, String> optionalParams) {
+ this.optionalParams = optionalParams;
+ }
+
public Map<String, JsonNode> getStatementParams() {
return statementParams;
}
@@ -366,31 +377,36 @@
throws IOException {
setHost(servlet.host(request));
setPath(servlet.servletPath(request));
- String contentType = HttpUtil.getContentTypeOnly(request);
+ setOptionalParams(optionalParams);
try {
- if (HttpUtil.ContentType.APPLICATION_JSON.equals(contentType)) {
- setParamFromJSON(request, optionalParams);
+ if (useRequestParameters(request)) {
+ setFromRequestParameters(request);
} else {
- setParamFromRequest(request, optionalParams);
+ setFromRequestBody(request);
}
} catch (JsonParseException | JsonMappingException e) {
throw new RuntimeDataException(ErrorCode.INVALID_REQ_JSON_VAL);
}
}
- private void setParamFromJSON(IServletRequest request, Map<String, String> optionalParameters) throws IOException {
+ private boolean useRequestParameters(IServletRequest request) {
+ String contentType = HttpUtil.getContentTypeOnly(request);
+ HttpMethod method = request.getHttpRequest().method();
+ return HttpMethod.GET.equals(method) || !HttpUtil.ContentType.APPLICATION_JSON.equals(contentType);
+ }
+
+ private void setFromRequestBody(IServletRequest request) throws IOException {
JsonNode jsonRequest = OBJECT_MAPPER.readTree(HttpUtil.getRequestBody(request));
setParams(jsonRequest, request.getHeader(HttpHeaders.ACCEPT), QueryServiceRequestParameters::getParameter);
setStatementParams(getOptStatementParameters(jsonRequest, jsonRequest.fieldNames(), JsonNode::get, v -> v));
- setJsonOptionalParameters(jsonRequest, optionalParameters);
+ setExtraParams(jsonRequest);
}
- private void setParamFromRequest(IServletRequest request, Map<String, String> optionalParameters)
- throws IOException {
+ private void setFromRequestParameters(IServletRequest request) throws IOException {
setParams(request, request.getHeader(HttpHeaders.ACCEPT), IServletRequest::getParameter);
setStatementParams(getOptStatementParameters(request, request.getParameterNames().iterator(),
IServletRequest::getParameter, OBJECT_MAPPER::readTree));
- setOptionalParameters(request, optionalParameters);
+ setExtraParams(request);
}
private <Req> void setParams(Req req, String acceptHeader, BiFunction<Req, String, String> valGetter)
@@ -421,13 +437,11 @@
setSignature(parseBoolean(req, Parameter.SIGNATURE.str(), valGetter, isSignature()));
}
- protected void setJsonOptionalParameters(JsonNode jsonRequest, Map<String, String> optionalParameters)
- throws HyracksDataException {
+ protected void setExtraParams(JsonNode jsonRequest) throws HyracksDataException {
// allows extensions to set extra parameters
}
- protected void setOptionalParameters(IServletRequest request, Map<String, String> optionalParameters)
- throws HyracksDataException {
+ protected void setExtraParams(IServletRequest request) throws HyracksDataException {
// allows extensions to set extra parameters
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
index 63678f8..feecbee 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
@@ -18,14 +18,12 @@
*/
package org.apache.asterix.api.http.server;
-import static org.apache.asterix.common.exceptions.ErrorCode.ASTERIX;
import static org.apache.asterix.common.exceptions.ErrorCode.INVALID_REQ_JSON_VAL;
import static org.apache.asterix.common.exceptions.ErrorCode.INVALID_REQ_PARAM_VAL;
import static org.apache.asterix.common.exceptions.ErrorCode.NO_STATEMENT_PROVIDED;
import static org.apache.asterix.common.exceptions.ErrorCode.REJECT_BAD_CLUSTER_STATE;
import static org.apache.asterix.common.exceptions.ErrorCode.REJECT_NODE_UNREGISTERED;
import static org.apache.asterix.common.exceptions.ErrorCode.REQUEST_TIMEOUT;
-import static org.apache.hyracks.api.exceptions.ErrorCode.HYRACKS;
import static org.apache.hyracks.api.exceptions.ErrorCode.JOB_REQUIREMENTS_EXCEED_CAPACITY;
import java.io.IOException;
@@ -88,6 +86,7 @@
import org.apache.asterix.translator.SessionOutput;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.application.IServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.exceptions.HyracksException;
import org.apache.hyracks.api.exceptions.Warning;
import org.apache.hyracks.control.common.controllers.CCConfig;
@@ -134,8 +133,13 @@
}
@Override
- protected void post(IServletRequest request, IServletResponse response) throws IOException {
- handleRequest(request, response);
+ protected final void get(IServletRequest request, IServletResponse response) throws IOException {
+ handleRequest(request, response, true);
+ }
+
+ @Override
+ protected final void post(IServletRequest request, IServletResponse response) throws IOException {
+ handleRequest(request, response, false);
}
@Override
@@ -147,18 +151,18 @@
response.setStatus(HttpResponseStatus.OK);
}
- protected static final class RequestExecutionState {
+ protected static class RequestExecutionState {
private long execStart = -1;
private long execEnd = -1;
- private ResultStatus resultStatus = ResultStatus.SUCCESS;
- private HttpResponseStatus httpResponseStatus = HttpResponseStatus.OK;
+ private ResultStatus resultStatus = ResultStatus.FATAL;
+ private HttpResponseStatus httpResponseStatus = HttpResponseStatus.INTERNAL_SERVER_ERROR;
public void setStatus(ResultStatus resultStatus, HttpResponseStatus httpResponseStatus) {
this.resultStatus = resultStatus;
this.httpResponseStatus = httpResponseStatus;
}
- ResultStatus getResultStatus() {
+ public ResultStatus getResultStatus() {
return resultStatus;
}
@@ -182,9 +186,19 @@
}
}
- long duration() {
+ public long duration() {
return execEnd - execStart;
}
+
+ protected StringBuilder append(StringBuilder sb) {
+ return sb.append("ResultStatus: ").append(resultStatus.str()).append(" HTTPStatus: ")
+ .append(String.valueOf(httpResponseStatus));
+ }
+
+ @Override
+ public String toString() {
+ return append(new StringBuilder()).toString();
+ }
}
private static SessionOutput createSessionOutput(PrintWriter resultWriter) {
@@ -196,7 +210,12 @@
}
protected void setRequestParam(IServletRequest request, QueryServiceRequestParameters param,
- Map<String, String> optionalParams) throws IOException, AlgebricksException {
+ Function<IServletRequest, Map<String, String>> optionalParamProvider, RequestExecutionState executionState)
+ throws IOException, AlgebricksException {
+ Map<String, String> optionalParams = null;
+ if (optionalParamProvider != null) {
+ optionalParams = optionalParamProvider.apply(request);
+ }
param.setParameters(this, request, optionalParams);
}
@@ -237,28 +256,28 @@
return "http://" + host + path + handlePath(delivery);
}
- private void handleRequest(IServletRequest request, IServletResponse response) throws IOException {
+ private void handleRequest(IServletRequest request, IServletResponse response, boolean forceReadOnly)
+ throws IOException {
final IRequestReference requestRef = receptionist.welcome(request);
long elapsedStart = System.nanoTime();
long errorCount = 1;
Stats stats = new Stats();
- RequestExecutionState execution = new RequestExecutionState();
List<Warning> warnings = new ArrayList<>();
Charset resultCharset = HttpUtil.setContentType(response, HttpUtil.ContentType.APPLICATION_JSON, request);
PrintWriter httpWriter = response.writer();
SessionOutput sessionOutput = createSessionOutput(httpWriter);
- QueryServiceRequestParameters param = newRequestParameters();
ResponsePrinter responsePrinter = new ResponsePrinter(sessionOutput);
ResultDelivery delivery = ResultDelivery.IMMEDIATE;
+ QueryServiceRequestParameters param = newRequestParameters();
+ RequestExecutionState executionState = newRequestExecutionState();
try {
// buffer the output until we are ready to set the status of the response message correctly
responsePrinter.begin();
- Map<String, String> optionalParams = null;
- if (optionalParamProvider != null) {
- optionalParams = optionalParamProvider.apply(request);
+ setRequestParam(request, param, optionalParamProvider, executionState);
+ if (forceReadOnly) {
+ param.setReadOnly(true);
}
- setRequestParam(request, param, optionalParams);
- LOGGER.info(() -> "handleRequest: " + LogRedactionUtil.userData(param.toString()));
+ LOGGER.info(() -> "handleRequest: " + LogRedactionUtil.statement(param.toString()));
delivery = param.getMode();
setSessionConfig(sessionOutput, param, delivery);
final ResultProperties resultProperties = new ResultProperties(delivery, param.getMaxResultReads());
@@ -269,27 +288,32 @@
if (param.isParseOnly()) {
ResultUtil.ParseOnlyResult parseOnlyResult = parseStatement(statementsText);
setAccessControlHeaders(request, response);
- response.setStatus(execution.getHttpStatus());
+ executionState.setStatus(ResultStatus.SUCCESS, HttpResponseStatus.OK);
+ response.setStatus(executionState.getHttpStatus());
responsePrinter.addResultPrinter(new ParseOnlyResultPrinter(parseOnlyResult));
} else {
Map<String, byte[]> statementParams = org.apache.asterix.app.translator.RequestParameters
.serializeParameterValues(param.getStatementParams());
setAccessControlHeaders(request, response);
- response.setStatus(execution.getHttpStatus());
stats.setProfileType(param.getProfileType());
- executeStatement(requestRef, statementsText, sessionOutput, resultProperties, stats, param, execution,
- optionalParams, statementParams, responsePrinter, warnings);
+ IStatementExecutor.StatementProperties statementProperties =
+ new IStatementExecutor.StatementProperties();
+ response.setStatus(HttpResponseStatus.OK);
+ executeStatement(requestRef, statementsText, sessionOutput, resultProperties, statementProperties,
+ stats, param, executionState, param.getOptionalParams(), statementParams, responsePrinter,
+ warnings);
+ executionState.setStatus(ResultStatus.SUCCESS, HttpResponseStatus.OK);
}
errorCount = 0;
} catch (Exception | TokenMgrError | org.apache.asterix.aqlplus.parser.TokenMgrError e) {
- handleExecuteStatementException(e, execution, param);
- response.setStatus(execution.getHttpStatus());
+ handleExecuteStatementException(e, executionState, param);
+ response.setStatus(executionState.getHttpStatus());
requestFailed(e, responsePrinter);
} finally {
- execution.finish();
+ executionState.finish();
}
responsePrinter.printResults();
- buildResponseFooters(elapsedStart, errorCount, stats, execution, resultCharset, responsePrinter, delivery);
+ buildResponseFooters(elapsedStart, errorCount, stats, executionState, resultCharset, responsePrinter, delivery);
responsePrinter.printFooters();
responsePrinter.end();
if (sessionOutput.out().checkError()) {
@@ -297,6 +321,10 @@
}
}
+ protected RequestExecutionState newRequestExecutionState() throws HyracksDataException {
+ return new RequestExecutionState();
+ }
+
protected void buildResponseHeaders(IRequestReference requestRef, SessionOutput sessionOutput,
QueryServiceRequestParameters param, ResponsePrinter responsePrinter, ResultDelivery delivery) {
responsePrinter.addHeaderPrinter(new RequestIdPrinter(requestRef.getUuid()));
@@ -313,7 +341,7 @@
}
protected void buildResponseResults(ResponsePrinter responsePrinter, SessionOutput sessionOutput,
- ExecutionPlans plans, List<Warning> warnings) {
+ ExecutionPlans plans, List<Warning> warnings) throws HyracksDataException {
responsePrinter.addResultPrinter(new PlansPrinter(plans, sessionOutput.config().getPlanFormat()));
if (!warnings.isEmpty()) {
List<ICodedMessage> codedWarnings = new ArrayList<>();
@@ -322,20 +350,21 @@
}
}
- protected void buildResponseFooters(long elapsedStart, long errorCount, Stats stats,
- RequestExecutionState execution, Charset resultCharset, ResponsePrinter responsePrinter,
+ protected ResponseMetrics buildResponseFooters(long elapsedStart, long errorCount, Stats stats,
+ RequestExecutionState executionState, Charset resultCharset, ResponsePrinter responsePrinter,
ResultDelivery delivery) {
if (ResultDelivery.ASYNC != delivery) {
// in case of ASYNC delivery, the status is printed by query translator
- responsePrinter.addFooterPrinter(new StatusPrinter(execution.getResultStatus()));
+ responsePrinter.addFooterPrinter(new StatusPrinter(executionState.getResultStatus()));
}
final ResponseMetrics metrics =
- ResponseMetrics.of(System.nanoTime() - elapsedStart, execution.duration(), stats.getCount(),
+ ResponseMetrics.of(System.nanoTime() - elapsedStart, executionState.duration(), stats.getCount(),
stats.getSize(), stats.getProcessedObjects(), errorCount, stats.getTotalWarningsCount());
responsePrinter.addFooterPrinter(new MetricsPrinter(metrics, resultCharset));
if (isPrintingProfile(stats)) {
responsePrinter.addFooterPrinter(new ProfilePrinter(stats.getJobProfile()));
}
+ return metrics;
}
protected void validateStatement(String statement) throws RuntimeDataException {
@@ -356,8 +385,9 @@
}
protected void executeStatement(IRequestReference requestReference, String statementsText,
- SessionOutput sessionOutput, ResultProperties resultProperties, Stats stats,
- QueryServiceRequestParameters param, RequestExecutionState execution,
+ SessionOutput sessionOutput, ResultProperties resultProperties,
+ IStatementExecutor.StatementProperties statementProperties, Stats stats,
+ QueryServiceRequestParameters param, RequestExecutionState executionState,
Map<String, String> optionalParameters, Map<String, byte[]> statementParameters,
ResponsePrinter responsePrinter, List<Warning> warnings) throws Exception {
IClusterManagementWork.ClusterState clusterState =
@@ -374,62 +404,57 @@
MetadataManager.INSTANCE.init();
IStatementExecutor translator = statementExecutorFactory.create((ICcApplicationContext) appCtx, statements,
sessionOutput, compilationProvider, componentProvider, responsePrinter);
- execution.start();
+ executionState.start();
Map<String, IAObject> stmtParams =
org.apache.asterix.app.translator.RequestParameters.deserializeParameterValues(statementParameters);
int stmtCategoryRestriction = org.apache.asterix.app.translator.RequestParameters
.getStatementCategoryRestrictionMask(param.isReadOnly());
- IRequestParameters requestParameters = new org.apache.asterix.app.translator.RequestParameters(requestReference,
- statementsText, getResultSet(), resultProperties, stats, null, param.getClientContextID(),
- optionalParameters, stmtParams, param.isMultiStatement(), stmtCategoryRestriction);
+ IRequestParameters requestParameters =
+ new org.apache.asterix.app.translator.RequestParameters(requestReference, statementsText,
+ getResultSet(), resultProperties, stats, statementProperties, null, param.getClientContextID(),
+ optionalParameters, stmtParams, param.isMultiStatement(), stmtCategoryRestriction);
translator.compileAndExecute(getHyracksClientConnection(), requestParameters);
- execution.end();
+ executionState.end();
translator.getWarnings(warnings, maxWarnings - warnings.size());
stats.updateTotalWarningsCount(parserTotalWarningsCount);
buildResponseResults(responsePrinter, sessionOutput, translator.getExecutionPlans(), warnings);
}
- protected void handleExecuteStatementException(Throwable t, RequestExecutionState state,
+ protected void handleExecuteStatementException(Throwable t, RequestExecutionState executionState,
QueryServiceRequestParameters param) {
if (t instanceof org.apache.asterix.aqlplus.parser.TokenMgrError || t instanceof TokenMgrError
|| t instanceof AlgebricksException) {
if (LOGGER.isDebugEnabled()) {
- LOGGER.debug("handleException: {}: {}", t.getMessage(), LogRedactionUtil.userData(param.toString()), t);
+ LOGGER.debug("handleException: {}: {}", t.getMessage(), LogRedactionUtil.statement(param.toString()),
+ t);
} else {
LOGGER.info(() -> "handleException: " + t.getMessage() + ": "
- + LogRedactionUtil.userData(param.toString()));
+ + LogRedactionUtil.statement(param.toString()));
}
- state.setStatus(ResultStatus.FATAL, HttpResponseStatus.BAD_REQUEST);
+ executionState.setStatus(ResultStatus.FATAL, HttpResponseStatus.BAD_REQUEST);
} else if (t instanceof HyracksException) {
HyracksException he = (HyracksException) t;
- switch (he.getComponent() + he.getErrorCode()) {
- case ASTERIX + REQUEST_TIMEOUT:
- LOGGER.info(() -> "handleException: request execution timed out: "
- + LogRedactionUtil.userData(param.toString()));
- state.setStatus(ResultStatus.TIMEOUT, HttpResponseStatus.OK);
- break;
- case ASTERIX + REJECT_BAD_CLUSTER_STATE:
- case ASTERIX + REJECT_NODE_UNREGISTERED:
- LOGGER.warn(() -> "handleException: " + he.getMessage() + ": "
- + LogRedactionUtil.userData(param.toString()));
- state.setStatus(ResultStatus.FATAL, HttpResponseStatus.SERVICE_UNAVAILABLE);
- break;
- case ASTERIX + INVALID_REQ_PARAM_VAL:
- case ASTERIX + INVALID_REQ_JSON_VAL:
- case ASTERIX + NO_STATEMENT_PROVIDED:
- case HYRACKS + JOB_REQUIREMENTS_EXCEED_CAPACITY:
- state.setStatus(ResultStatus.FATAL, HttpResponseStatus.BAD_REQUEST);
- break;
- default:
- LOGGER.warn(() -> "handleException: unexpected exception " + he.getMessage() + ": "
- + LogRedactionUtil.userData(param.toString()), he);
- state.setStatus(ResultStatus.FATAL, HttpResponseStatus.INTERNAL_SERVER_ERROR);
- break;
+ // TODO(mblow): reconsolidate
+ if (he.matchesAny(INVALID_REQ_PARAM_VAL, INVALID_REQ_JSON_VAL, NO_STATEMENT_PROVIDED,
+ JOB_REQUIREMENTS_EXCEED_CAPACITY)) {
+ executionState.setStatus(ResultStatus.FATAL, HttpResponseStatus.BAD_REQUEST);
+ } else if (he.matches(REQUEST_TIMEOUT)) {
+ LOGGER.info(() -> "handleException: request execution timed out: "
+ + LogRedactionUtil.userData(param.toString()));
+ executionState.setStatus(ResultStatus.TIMEOUT, HttpResponseStatus.OK);
+ } else if (he.matchesAny(REJECT_BAD_CLUSTER_STATE, REJECT_NODE_UNREGISTERED)) {
+ LOGGER.warn(() -> "handleException: " + he.getMessage() + ": "
+ + LogRedactionUtil.userData(param.toString()));
+ executionState.setStatus(ResultStatus.FATAL, HttpResponseStatus.SERVICE_UNAVAILABLE);
+ } else {
+ LOGGER.warn(() -> "handleException: unexpected exception " + he.getMessage() + ": "
+ + LogRedactionUtil.userData(param.toString()), he);
+ executionState.setStatus(ResultStatus.FATAL, HttpResponseStatus.INTERNAL_SERVER_ERROR);
}
} else {
LOGGER.warn(() -> "handleException: unexpected exception: " + LogRedactionUtil.userData(param.toString()),
t);
- state.setStatus(ResultStatus.FATAL, HttpResponseStatus.INTERNAL_SERVER_ERROR);
+ executionState.setStatus(ResultStatus.FATAL, HttpResponseStatus.INTERNAL_SERVER_ERROR);
}
}
@@ -464,7 +489,7 @@
return new QueryServiceRequestParameters();
}
- private static boolean isPrintingProfile(IStatementExecutor.Stats stats) {
+ protected static boolean isPrintingProfile(IStatementExecutor.Stats stats) {
return stats.getProfileType() == Stats.ProfileType.FULL && stats.getJobProfile() != null;
}
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RebalanceApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RebalanceApiServlet.java
index 320c7aa..43c25f2 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RebalanceApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RebalanceApiServlet.java
@@ -21,6 +21,7 @@
import static org.apache.asterix.api.http.server.ServletConstants.HYRACKS_CONNECTION_ATTR;
import java.io.PrintWriter;
+import java.nio.charset.StandardCharsets;
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Arrays;
@@ -106,7 +107,11 @@
String dataverseName = request.getParameter("dataverseName");
String datasetName = request.getParameter("datasetName");
String nodes = request.getParameter("nodes");
-
+ boolean forceRebalance = true;
+ String force = request.getParameter("force");
+ if (force != null) {
+ forceRebalance = Boolean.parseBoolean(force);
+ }
// Parses and check target nodes.
if (nodes == null) {
sendResponse(response, HttpResponseStatus.BAD_REQUEST, "nodes are not given");
@@ -132,7 +137,8 @@
return;
}
// Schedules a rebalance task and wait for its completion.
- CountDownLatch terminated = scheduleRebalance(dataverseName, datasetName, targetNodes, response);
+ CountDownLatch terminated =
+ scheduleRebalance(dataverseName, datasetName, targetNodes, response, forceRebalance);
terminated.await();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
@@ -155,10 +161,10 @@
// Schedules a rebalance task.
private synchronized CountDownLatch scheduleRebalance(String dataverseName, String datasetName,
- String[] targetNodes, IServletResponse response) {
+ String[] targetNodes, IServletResponse response, boolean force) {
CountDownLatch terminated = new CountDownLatch(1);
- Future<Void> task =
- executor.submit(() -> doRebalance(dataverseName, datasetName, targetNodes, response, terminated));
+ Future<Void> task = executor
+ .submit(() -> doRebalance(dataverseName, datasetName, targetNodes, response, terminated, force));
rebalanceTasks.add(task);
rebalanceFutureTerminated.add(terminated);
return terminated;
@@ -166,10 +172,10 @@
// Performs the actual rebalance.
private Void doRebalance(String dataverseName, String datasetName, String[] targetNodes, IServletResponse response,
- CountDownLatch terminated) {
+ CountDownLatch terminated, boolean force) {
try {
// Sets the content type.
- HttpUtil.setContentType(response, HttpUtil.ContentType.APPLICATION_JSON, HttpUtil.Encoding.UTF8);
+ HttpUtil.setContentType(response, HttpUtil.ContentType.APPLICATION_JSON, StandardCharsets.UTF_8);
if (datasetName == null) {
// Rebalances datasets in a given dataverse or all non-metadata datasets.
@@ -178,11 +184,11 @@
for (Dataset dataset : datasets) {
// By the time rebalanceDataset(...) is called, the dataset could have been dropped.
// If that's the case, rebalanceDataset(...) would be a no-op.
- rebalanceDataset(dataset.getDataverseName(), dataset.getDatasetName(), targetNodes);
+ rebalanceDataset(dataset.getDataverseName(), dataset.getDatasetName(), targetNodes, force);
}
} else {
// Rebalances a given dataset from its current locations to the target nodes.
- rebalanceDataset(dataverseName, datasetName, targetNodes);
+ rebalanceDataset(dataverseName, datasetName, targetNodes, force);
}
// Sends response.
@@ -242,7 +248,8 @@
}
// Rebalances a given dataset.
- private void rebalanceDataset(String dataverseName, String datasetName, String[] targetNodes) throws Exception {
+ private void rebalanceDataset(String dataverseName, String datasetName, String[] targetNodes, boolean force)
+ throws Exception {
IHyracksClientConnection hcc = (IHyracksClientConnection) ctx.get(HYRACKS_CONNECTION_ATTR);
MetadataProvider metadataProvider = new MetadataProvider(appCtx, null);
try {
@@ -254,7 +261,7 @@
lockManager.acquireDatasetExclusiveModificationLock(metadataProvider.getLocks(),
dataverseName + '.' + datasetName);
RebalanceUtil.rebalance(dataverseName, datasetName, new LinkedHashSet<>(Arrays.asList(targetNodes)),
- metadataProvider, hcc, NoOpDatasetRebalanceCallback.INSTANCE);
+ metadataProvider, hcc, NoOpDatasetRebalanceCallback.INSTANCE, force);
} finally {
activeNotificationHandler.resume(metadataProvider);
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ResultUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ResultUtil.java
index 63da9ce..045d64c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ResultUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ResultUtil.java
@@ -48,6 +48,7 @@
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.prettyprint.AlgebricksAppendable;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IFormattedException;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -190,12 +191,16 @@
public static Throwable getRootCause(Throwable cause) {
Throwable currentCause = cause;
+ Throwable rootFormattedEx = cause instanceof IFormattedException ? cause : null;
Throwable nextCause = cause.getCause();
while (nextCause != null && nextCause != currentCause) {
currentCause = nextCause;
+ if (currentCause instanceof IFormattedException) {
+ rootFormattedEx = currentCause;
+ }
nextCause = nextCause.getCause();
}
- return currentCause;
+ return rootFormattedEx != null ? rootFormattedEx : currentCause;
}
/**
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java
index 1cdf6f7..4fe6582 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java
@@ -136,8 +136,8 @@
final RequestReference requestReference =
RequestReference.of(UUID.randomUUID().toString(), "CC", System.currentTimeMillis());
final IRequestParameters requestParameters = new RequestParameters(requestReference, statement, null,
- new ResultProperties(IStatementExecutor.ResultDelivery.IMMEDIATE), new IStatementExecutor.Stats(), null,
- null, null, statementParams, true);
+ new ResultProperties(IStatementExecutor.ResultDelivery.IMMEDIATE), new IStatementExecutor.Stats(),
+ new IStatementExecutor.StatementProperties(), null, null, null, statementParams, true);
translator.compileAndExecute(hcc, requestParameters);
executionPlans = translator.getExecutionPlans();
writer.flush();
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java
index 882afc5..39ebdf7 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/ActiveEntityEventsListener.java
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.app.active;
+import static org.apache.asterix.common.exceptions.ErrorCode.ACTIVE_ENTITY_NOT_RUNNING;
+
import java.util.ArrayList;
import java.util.Arrays;
import java.util.EnumSet;
@@ -93,7 +95,7 @@
protected ActivityState prevState;
protected JobId jobId;
protected volatile long statsTimestamp;
- protected String stats;
+ protected volatile String stats;
protected volatile boolean isFetchingStats;
protected int numRegistered;
protected int numDeRegistered;
@@ -292,12 +294,17 @@
@Override
public void refreshStats(long timeout) throws HyracksDataException {
LOGGER.log(level, "refreshStats called");
+ // first check state & if we are fetching outside of the lock- in the event we are recovering it may take some
+ // time to obtain the lock...
+ ensureRunning();
+ if (isFetchingStats) {
+ LOGGER.log(level, "returning immediately since fetchingStats = " + isFetchingStats);
+ return;
+ }
synchronized (this) {
- if (state != ActivityState.RUNNING) {
- LOGGER.log(level, "returning immediately since state = " + state);
- notifySubscribers(statsUpdatedEvent);
- return;
- } else if (isFetchingStats) {
+ // now that we have the lock, again verify the state & ensure we are not already fetching new stats
+ ensureRunning();
+ if (isFetchingStats) {
LOGGER.log(level, "returning immediately since fetchingStats = " + isFetchingStats);
return;
} else {
@@ -323,6 +330,12 @@
isFetchingStats = false;
}
+ protected void ensureRunning() throws RuntimeDataException {
+ if (state != ActivityState.RUNNING) {
+ throw new RuntimeDataException(ACTIVE_ENTITY_NOT_RUNNING, runtimeName, String.valueOf(state).toLowerCase());
+ }
+ }
+
protected synchronized void notifySubscribers(ActiveEvent event) {
notifyAll();
Iterator<IActiveEntityEventSubscriber> it = subscribers.iterator();
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/RecoveryTask.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/RecoveryTask.java
index a1989fc..340c454 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/RecoveryTask.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/active/RecoveryTask.java
@@ -106,10 +106,10 @@
protected Void doRecover(IRetryPolicy policy) throws AlgebricksException, InterruptedException {
LOGGER.log(level, "Actual Recovery task has started");
- Exception failure = null;
- while (policy.retry(failure)) {
+ Exception failure;
+ do {
synchronized (listener) {
- while (!cancelRecovery && clusterStateManager.getState() != ClusterState.ACTIVE) {
+ while (!cancelRecovery && !canStartRecovery()) {
listener.wait();
}
if (cancelRecovery) {
@@ -139,7 +139,7 @@
} finally {
releaseRecoveryLocks(metadataProvider);
}
- }
+ } while (policy.retry(failure));
// Recovery task is essntially over now either through failure or through cancellation(stop)
synchronized (listener) {
listener.notifyAll();
@@ -198,4 +198,9 @@
protected void releasePostRecoveryLocks() {
metadataProvider.getLocks().reset();
}
+
+ private boolean canStartRecovery() {
+ return clusterStateManager.getState() == ClusterState.ACTIVE
+ || clusterStateManager.getState() == ClusterState.REBALANCE_REQUIRED;
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalLibraryUtils.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalLibraryUtils.java
index a989941..c185340 100755
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalLibraryUtils.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/external/ExternalLibraryUtils.java
@@ -34,9 +34,9 @@
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.external.IDataSourceAdapter;
import org.apache.asterix.common.functions.FunctionSignature;
import org.apache.asterix.common.library.ILibraryManager;
-import org.apache.asterix.external.api.IDataSourceAdapter;
import org.apache.asterix.external.dataset.adapter.AdapterIdentifier;
import org.apache.asterix.external.library.ExternalLibrary;
import org.apache.asterix.external.library.LibraryAdapter;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
index 94080da..2eced12 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
@@ -146,10 +146,12 @@
compilationProvider, storageComponentProvider, new ResponsePrinter(sessionOutput));
final IStatementExecutor.Stats stats = new IStatementExecutor.Stats();
stats.setProfileType(profileType);
+ final IStatementExecutor.StatementProperties statementProperties =
+ new IStatementExecutor.StatementProperties();
Map<String, IAObject> stmtParams = RequestParameters.deserializeParameterValues(statementParameters);
final IRequestParameters requestParameters = new RequestParameters(requestReference, statementsText, null,
- resultProperties, stats, outMetadata, clientContextID, optionalParameters, stmtParams,
- multiStatement, statementCategoryRestrictionMask);
+ resultProperties, stats, statementProperties, outMetadata, clientContextID, optionalParameters,
+ stmtParams, multiStatement, statementCategoryRestrictionMask);
translator.compileAndExecute(ccApp.getHcc(), requestParameters);
translator.getWarnings(warnings, maxWarnings - warnings.size());
stats.updateTotalWarningsCount(parserTotalWarningsCount);
@@ -157,6 +159,7 @@
responseMsg.setResult(outWriter.toString());
responseMsg.setMetadata(outMetadata);
responseMsg.setStats(stats);
+ responseMsg.setStatementProperties(statementProperties);
responseMsg.setExecutionPlans(translator.getExecutionPlans());
responseMsg.setWarnings(warnings);
} catch (AlgebricksException | HyracksException | TokenMgrError
@@ -201,6 +204,6 @@
@Override
public String toString() {
return String.format("%s(id=%s, from=%s): %s", getClass().getSimpleName(), requestMessageId, requestNodeId,
- LogRedactionUtil.userData(statementsText));
+ LogRedactionUtil.statement(statementsText));
}
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementResponseMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementResponseMessage.java
index 58898cf..2cdede1 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementResponseMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementResponseMessage.java
@@ -41,6 +41,8 @@
private IStatementExecutor.Stats stats;
+ private IStatementExecutor.StatementProperties statementProperties;
+
private Throwable error;
private ExecutionPlans executionPlans;
@@ -92,6 +94,14 @@
this.stats = stats;
}
+ public IStatementExecutor.StatementProperties getStatementProperties() {
+ return statementProperties;
+ }
+
+ public void setStatementProperties(IStatementExecutor.StatementProperties statementProperties) {
+ this.statementProperties = statementProperties;
+ }
+
public ExecutionPlans getExecutionPlans() {
return executionPlans;
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
index 13ca95d..7829b75 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
@@ -444,7 +444,8 @@
// our client socket factory when SSL is enabled
if (networkSecurityManager.getConfiguration().isSslEnabled()) {
final RMIServerFactory serverSocketFactory = new RMIServerFactory(networkSecurityManager);
- final RMIClientFactory clientSocketFactory = new RMIClientFactory(true);
+ final RMIClientFactory clientSocketFactory =
+ new RMIClientFactory(networkSecurityManager.getConfiguration());
metadataNodeStub = (IMetadataNode) UnicastRemoteObject.exportObject(MetadataNode.INSTANCE,
getMetadataProperties().getMetadataPort(), clientSocketFactory, serverSocketFactory);
} else {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
index 3fce4d8..718e4b3 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
@@ -173,15 +173,14 @@
deleteRecoveryTemporaryFiles();
//get active partitions on this node
- replayPartitionsLogs(partitions, logMgr.getLogReader(true), lowWaterMarkLSN);
+ replayPartitionsLogs(partitions, logMgr.getLogReader(true), lowWaterMarkLSN, true);
}
- @Override
- public synchronized void replayPartitionsLogs(Set<Integer> partitions, ILogReader logReader, long lowWaterMarkLSN)
- throws IOException, ACIDException {
+ public synchronized void replayPartitionsLogs(Set<Integer> partitions, ILogReader logReader, long lowWaterMarkLSN,
+ boolean closeOnFlushRedo) throws IOException, ACIDException {
try {
Set<Long> winnerJobSet = startRecoverysAnalysisPhase(partitions, logReader, lowWaterMarkLSN);
- startRecoveryRedoPhase(partitions, logReader, lowWaterMarkLSN, winnerJobSet);
+ startRecoveryRedoPhase(partitions, logReader, lowWaterMarkLSN, winnerJobSet, closeOnFlushRedo);
} finally {
logReader.close();
deleteRecoveryTemporaryFiles();
@@ -277,7 +276,7 @@
}
private synchronized void startRecoveryRedoPhase(Set<Integer> partitions, ILogReader logReader,
- long lowWaterMarkLSN, Set<Long> winnerTxnSet) throws IOException, ACIDException {
+ long lowWaterMarkLSN, Set<Long> winnerTxnSet, boolean closeOnFlushRedo) throws IOException, ACIDException {
int redoCount = 0;
long txnId = 0;
@@ -299,6 +298,7 @@
TxnEntityId tempKeyTxnEntityId = new TxnEntityId(-1, -1, -1, null, -1, false);
ILogRecord logRecord = null;
+ Set<Integer> flushRedoDatasets = new HashSet<>();
try {
logReader.setPosition(lowWaterMarkLSN);
logRecord = logReader.next();
@@ -409,6 +409,7 @@
&& !index.isCurrentMutableComponentEmpty()) {
// schedule flush
redoFlush(index, logRecord);
+ flushRedoDatasets.add(datasetId);
redoCount++;
} else {
// TODO: update checkpoint file?
@@ -441,6 +442,11 @@
for (long r : resourceIdList) {
datasetLifecycleManager.close(resourcesMap.get(r).getPath());
}
+ if (closeOnFlushRedo) {
+ // close datasets of indexes to ensure any cached state that might've been changed by recovery is cleared
+ // e.g. when redoing a flush, the component id generator needs to be reinitialized
+ datasetLifecycleManager.closeDatasets(flushRedoDatasets);
+ }
}
}
@@ -525,7 +531,7 @@
if (minLSN < readableSmallestLSN) {
minLSN = readableSmallestLSN;
}
- replayPartitionsLogs(partitions, logMgr.getLogReader(true), minLSN);
+ replayPartitionsLogs(partitions, logMgr.getLogReader(true), minLSN, false);
if (flush) {
appCtx.getDatasetLifecycleManager().flushAllDatasets();
}
@@ -787,7 +793,7 @@
} catch (HyracksDataException hde) {
// Since we're undoing according the write-ahead log, the actual upserting tuple
// might not have been written to memory yet.
- if (hde.getErrorCode() != ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
+ if (!hde.matches(ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)) {
throw hde;
}
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index ce1a354..1befe29 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -85,12 +85,15 @@
import org.apache.asterix.external.indexing.IndexingConstants;
import org.apache.asterix.external.operators.FeedIntakeOperatorNodePushable;
import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
import org.apache.asterix.formats.nontagged.TypeTraitProvider;
import org.apache.asterix.lang.common.base.IReturningStatement;
import org.apache.asterix.lang.common.base.IRewriterFactory;
import org.apache.asterix.lang.common.base.IStatementRewriter;
import org.apache.asterix.lang.common.base.Statement;
import org.apache.asterix.lang.common.expression.IndexedTypeExpression;
+import org.apache.asterix.lang.common.expression.TypeExpression;
+import org.apache.asterix.lang.common.expression.TypeReferenceExpression;
import org.apache.asterix.lang.common.statement.CompactStatement;
import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
import org.apache.asterix.lang.common.statement.CreateDataverseStatement;
@@ -196,6 +199,7 @@
import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
import org.apache.hyracks.api.exceptions.SourceLocation;
import org.apache.hyracks.api.exceptions.Warning;
import org.apache.hyracks.api.io.FileSplit;
@@ -271,6 +275,10 @@
return functionDecls;
}
+ public IWarningCollector getWarningCollector() {
+ return warningCollector;
+ }
+
@Override
public void compileAndExecute(IHyracksClientConnection hcc, IRequestParameters requestParameters) throws Exception {
validateStatements(statements, requestParameters.isMultiStatement(),
@@ -288,6 +296,7 @@
final ResultDelivery resultDelivery = requestParameters.getResultProperties().getDelivery();
final long maxResultReads = requestParameters.getResultProperties().getMaxReads();
final Stats stats = requestParameters.getStats();
+ final StatementProperties statementProperties = requestParameters.getStatementProperties();
final ResultMetadata outMetadata = requestParameters.getOutMetadata();
final Map<String, IAObject> stmtParams = requestParameters.getStatementParameters();
warningCollector.setMaxWarnings(sessionConfig.getMaxWarnings());
@@ -304,7 +313,9 @@
metadataProvider.setWriterFactory(writerFactory);
metadataProvider.setResultSerializerFactoryProvider(resultSerializerFactoryProvider);
metadataProvider.setOutputFile(outputFile);
- switch (stmt.getKind()) {
+ Statement.Kind kind = stmt.getKind();
+ statementProperties.setKind(kind);
+ switch (kind) {
case SET:
handleSetStatement(stmt, config);
break;
@@ -414,12 +425,13 @@
// No op
break;
case EXTENSION:
- ((ExtensionStatement) stmt).handle(hcc, this, requestParameters, metadataProvider,
- resultSetIdCounter);
+ final ExtensionStatement extStmt = (ExtensionStatement) stmt;
+ statementProperties.setName(extStmt.getName());
+ extStmt.handle(hcc, this, requestParameters, metadataProvider, resultSetIdCounter);
break;
default:
throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, stmt.getSourceLocation(),
- "Unexpected statement: " + stmt.getKind());
+ "Unexpected statement: " + kind);
}
}
} finally {
@@ -479,7 +491,9 @@
protected void handleCreateDataverseStatement(MetadataProvider metadataProvider, Statement stmt,
IRequestParameters requestParameters) throws Exception {
CreateDataverseStatement stmtCreateDataverse = (CreateDataverseStatement) stmt;
+ SourceLocation sourceLoc = stmtCreateDataverse.getSourceLocation();
String dvName = stmtCreateDataverse.getDataverseName().getValue();
+ validateDataverseName(dvName, sourceLoc);
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
metadataProvider.setMetadataTxnContext(mdTxnCtx);
lockManager.acquireDataverseReadLock(metadataProvider.getLocks(), dvName);
@@ -557,11 +571,60 @@
SourceLocation sourceLoc = dd.getSourceLocation();
String dataverseName = getActiveDataverse(dd.getDataverse());
String datasetName = dd.getName().getValue();
+ validateDatabaseObjectName(datasetName, sourceLoc);
DatasetType dsType = dd.getDatasetType();
- String itemTypeDataverseName = getActiveDataverse(dd.getItemTypeDataverse());
- String itemTypeName = dd.getItemTypeName().getValue();
- String metaItemTypeDataverseName = getActiveDataverse(dd.getMetaItemTypeDataverse());
- String metaItemTypeName = dd.getMetaItemTypeName().getValue();
+ TypeExpression itemTypeExpr = dd.getItemType();
+ String itemTypeDataverseName, itemTypeName;
+ boolean itemTypeAnonymous;
+ switch (itemTypeExpr.getTypeKind()) {
+ case TYPEREFERENCE:
+ TypeReferenceExpression itemTypeRefExpr = (TypeReferenceExpression) itemTypeExpr;
+ Identifier itemTypeDataverseIdent = itemTypeRefExpr.getIdent().first;
+ itemTypeDataverseName = itemTypeDataverseIdent != null && itemTypeDataverseIdent.getValue() != null
+ ? itemTypeDataverseIdent.getValue() : dataverseName;
+ itemTypeName = itemTypeRefExpr.getIdent().second.getValue();
+ itemTypeAnonymous = false;
+ break;
+ case RECORD:
+ itemTypeDataverseName = dataverseName;
+ itemTypeName = DatasetUtil.createInlineTypeName(datasetName, false);
+ itemTypeAnonymous = true;
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
+ String.valueOf(itemTypeExpr.getTypeKind()));
+ }
+ String itemTypeFullyQualifiedName = itemTypeDataverseName + '.' + itemTypeName;
+
+ TypeExpression metaItemTypeExpr = dd.getMetaItemType();
+ String metaItemTypeDataverseName = null, metaItemTypeName = null, metaItemTypeFullyQualifiedName;
+ boolean metaItemTypeAnonymous;
+ if (metaItemTypeExpr != null) {
+ switch (metaItemTypeExpr.getTypeKind()) {
+ case TYPEREFERENCE:
+ TypeReferenceExpression metaItemTypeRefExpr = (TypeReferenceExpression) metaItemTypeExpr;
+ Identifier metaItemTypeDataverseIdent = metaItemTypeRefExpr.getIdent().first;
+ metaItemTypeDataverseName =
+ metaItemTypeDataverseIdent != null && metaItemTypeDataverseIdent.getValue() != null
+ ? metaItemTypeDataverseIdent.getValue() : dataverseName;
+ metaItemTypeName = metaItemTypeRefExpr.getIdent().second.getValue();
+ metaItemTypeAnonymous = false;
+ break;
+ case RECORD:
+ metaItemTypeDataverseName = dataverseName;
+ metaItemTypeName = DatasetUtil.createInlineTypeName(datasetName, true);
+ metaItemTypeAnonymous = true;
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
+ String.valueOf(metaItemTypeExpr.getTypeKind()));
+ }
+ metaItemTypeFullyQualifiedName = metaItemTypeDataverseName + '.' + metaItemTypeName;
+ } else {
+ metaItemTypeFullyQualifiedName = null;
+ metaItemTypeAnonymous = true; // doesn't matter
+ }
+
Identifier ngNameId = dd.getNodegroupName();
String nodegroupName = ngNameId == null ? null : ngNameId.getValue();
String compactionPolicy = dd.getCompactionPolicy();
@@ -572,13 +635,19 @@
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
boolean bActiveTxn = true;
metadataProvider.setMetadataTxnContext(mdTxnCtx);
- MetadataLockUtil.createDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName,
- itemTypeDataverseName, itemTypeDataverseName + "." + itemTypeName, metaItemTypeDataverseName,
- metaItemTypeDataverseName + "." + metaItemTypeName, nodegroupName, compactionPolicy,
- dataverseName + "." + datasetName, defaultCompactionPolicy);
+ createDatasetBegin(metadataProvider, dataverseName, dataverseName + "." + datasetName, itemTypeDataverseName,
+ itemTypeAnonymous, itemTypeFullyQualifiedName, metaItemTypeDataverseName,
+ metaItemTypeFullyQualifiedName, metaItemTypeAnonymous, nodegroupName, compactionPolicy,
+ defaultCompactionPolicy, dd);
Dataset dataset = null;
try {
- IDatasetDetails datasetDetails = null;
+ // Check if the dataverse exists
+ Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
+ if (dv == null) {
+ throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc, dataverseName);
+ }
+
+ IDatasetDetails datasetDetails;
Dataset ds = metadataProvider.findDataset(dataverseName, datasetName);
if (ds != null) {
if (dd.getIfNotExists()) {
@@ -588,10 +657,27 @@
throw new CompilationException(ErrorCode.DATASET_EXISTS, sourceLoc, datasetName, dataverseName);
}
}
- Datatype dt = MetadataManager.INSTANCE.getDatatype(metadataProvider.getMetadataTxnContext(),
- itemTypeDataverseName, itemTypeName);
- if (dt == null) {
- throw new CompilationException(ErrorCode.UNKNOWN_TYPE, sourceLoc, itemTypeName);
+ Datatype itemTypeEntity;
+ IAType itemType;
+ switch (itemTypeExpr.getTypeKind()) {
+ case TYPEREFERENCE:
+ itemTypeEntity = metadataProvider.findTypeEntity(itemTypeDataverseName, itemTypeName);
+ if (itemTypeEntity == null || itemTypeEntity.getIsAnonymous()) {
+ // anonymous types cannot be referred from CREATE DATASET
+ throw new AsterixException(ErrorCode.UNKNOWN_TYPE, sourceLoc, itemTypeFullyQualifiedName);
+ }
+ itemType = itemTypeEntity.getDatatype();
+ validateDatasetItemType(dsType, itemType, false, sourceLoc);
+ break;
+ case RECORD:
+ itemType = translateType(itemTypeDataverseName, itemTypeName, itemTypeExpr, mdTxnCtx);
+ validateDatasetItemType(dsType, itemType, false, sourceLoc);
+ itemTypeEntity = new Datatype(itemTypeDataverseName, itemTypeName, itemType, true);
+ MetadataManager.INSTANCE.addDatatype(mdTxnCtx, itemTypeEntity);
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
+ String.valueOf(itemTypeExpr.getTypeKind()));
}
String ngName = ngNameId != null ? ngNameId.getValue()
: configureNodegroupForDataset(appCtx, dd.getHints(), dataverseName, datasetName, metadataProvider,
@@ -603,21 +689,33 @@
} else {
validateCompactionPolicy(compactionPolicy, compactionPolicyProperties, mdTxnCtx, false, sourceLoc);
}
- switch (dd.getDatasetType()) {
+ switch (dsType) {
case INTERNAL:
- IAType itemType = dt.getDatatype();
- if (itemType.getTypeTag() != ATypeTag.OBJECT) {
- throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
- "Dataset type has to be a record type.");
- }
-
IAType metaItemType = null;
- if (metaItemTypeDataverseName != null && metaItemTypeName != null) {
- metaItemType = metadataProvider.findType(metaItemTypeDataverseName, metaItemTypeName);
- }
- if (metaItemType != null && metaItemType.getTypeTag() != ATypeTag.OBJECT) {
- throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
- "Dataset meta type has to be a record type.");
+ if (metaItemTypeExpr != null) {
+ switch (metaItemTypeExpr.getTypeKind()) {
+ case TYPEREFERENCE:
+ Datatype metaItemTypeEntity =
+ metadataProvider.findTypeEntity(metaItemTypeDataverseName, metaItemTypeName);
+ if (metaItemTypeEntity == null || metaItemTypeEntity.getIsAnonymous()) {
+ // anonymous types cannot be referred from CREATE DATASET
+ throw new AsterixException(ErrorCode.UNKNOWN_TYPE, sourceLoc,
+ metaItemTypeFullyQualifiedName);
+ }
+ metaItemType = metaItemTypeEntity.getDatatype();
+ validateDatasetItemType(dsType, metaItemType, true, sourceLoc);
+ break;
+ case RECORD:
+ metaItemType = translateType(metaItemTypeDataverseName, metaItemTypeName,
+ metaItemTypeExpr, mdTxnCtx);
+ validateDatasetItemType(dsType, metaItemType, true, sourceLoc);
+ MetadataManager.INSTANCE.addDatatype(mdTxnCtx,
+ new Datatype(metaItemTypeDataverseName, metaItemTypeName, metaItemType, true));
+ break;
+ default:
+ throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
+ String.valueOf(metaItemTypeExpr.getTypeKind()));
+ }
}
ARecordType metaRecType = (ARecordType) metaItemType;
@@ -646,15 +744,18 @@
keySourceIndicators, partitioningTypes, autogenerated, filterField);
break;
case EXTERNAL:
- String adapter = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getAdapter();
- Map<String, String> properties = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getProperties();
-
- datasetDetails =
- new ExternalDatasetDetails(adapter, properties, new Date(), TransactionState.COMMIT);
+ ExternalDetailsDecl externalDetails = (ExternalDetailsDecl) dd.getDatasetDetailsDecl();
+ Map<String, String> properties = createExternalDatasetProperties(dataverseName, dd, itemTypeEntity,
+ metadataProvider, mdTxnCtx);
+ ExternalDataUtils.normalize(properties);
+ ExternalDataUtils.validate(properties);
+ validateExternalDatasetProperties(externalDetails, properties, dd.getSourceLocation(), mdTxnCtx);
+ datasetDetails = new ExternalDatasetDetails(externalDetails.getAdapter(), properties, new Date(),
+ TransactionState.COMMIT);
break;
default:
throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
- "Unknown dataset type " + dd.getDatasetType());
+ "Unknown dataset type " + dsType);
}
// #. initialize DatasetIdFactory if it is not initialized.
@@ -668,7 +769,7 @@
datasetDetails, dd.getHints(), dsType, DatasetIdFactory.generateDatasetId(),
MetadataUtil.PENDING_ADD_OP, compressionScheme);
MetadataManager.INSTANCE.addDataset(metadataProvider.getMetadataTxnContext(), dataset);
- if (dd.getDatasetType() == DatasetType.INTERNAL) {
+ if (dsType == DatasetType.INTERNAL) {
JobSpecification jobSpec = DatasetUtil.createDatasetJobSpec(dataset, metadataProvider);
// #. make metadataTxn commit before calling runJob.
@@ -739,6 +840,32 @@
}
}
+ protected void createDatasetBegin(MetadataProvider metadataProvider, String dataverseName,
+ String datasetFullyQualifiedName, String itemTypeDataverseName, boolean itemTypeAnonymous,
+ String itemTypeFullyQualifiedName, String metaItemTypeDataverseName, String metaItemTypeFullyQualifiedName,
+ boolean metaItemTypeAnonymous, String nodegroupName, String compactionPolicy,
+ boolean defaultCompactionPolicy, DatasetDecl dd) throws AlgebricksException {
+ MetadataLockUtil.createDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName,
+ itemTypeDataverseName, itemTypeFullyQualifiedName, itemTypeAnonymous, metaItemTypeDataverseName,
+ metaItemTypeFullyQualifiedName, metaItemTypeAnonymous, nodegroupName, compactionPolicy,
+ datasetFullyQualifiedName, defaultCompactionPolicy);
+ }
+
+ protected void validateDatasetItemType(DatasetType datasetType, IAType itemType, boolean isMetaItemType,
+ SourceLocation sourceLoc) throws AlgebricksException {
+ if (itemType.getTypeTag() != ATypeTag.OBJECT) {
+ throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
+ String.format("Dataset %s has to be a record type.", isMetaItemType ? "meta type" : "type"));
+ }
+ }
+
+ protected Map<String, String> createExternalDatasetProperties(String dataverseName, DatasetDecl dd,
+ Datatype itemType, MetadataProvider metadataProvider, MetadataTransactionContext mdTxnCtx)
+ throws AlgebricksException {
+ ExternalDetailsDecl externalDetails = (ExternalDetailsDecl) dd.getDatasetDetailsDecl();
+ return externalDetails.getProperties();
+ }
+
protected static void validateIfResourceIsActiveInFeed(ICcApplicationContext appCtx, Dataset dataset,
SourceLocation sourceLoc) throws CompilationException {
ActiveNotificationHandler activeEventHandler =
@@ -787,22 +914,33 @@
String dataverseName = getActiveDataverse(stmtCreateIndex.getDataverseName());
String datasetName = stmtCreateIndex.getDatasetName().getValue();
String indexName = stmtCreateIndex.getIndexName().getValue();
+ validateDatabaseObjectName(indexName, sourceLoc);
+ IndexType indexType = stmtCreateIndex.getIndexType();
List<Integer> keySourceIndicators = stmtCreateIndex.getFieldSourceIndicators();
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
metadataProvider.setMetadataTxnContext(mdTxnCtx);
String datasetFullyQualifiedName = dataverseName + "." + datasetName;
boolean isSecondaryPrimary = stmtCreateIndex.getFieldExprs().isEmpty();
- Dataset ds = null;
- Index index = null;
+ Dataset ds;
+ Index index;
MetadataLockUtil.createIndexBegin(lockManager, metadataProvider.getLocks(), dataverseName,
datasetFullyQualifiedName);
try {
+ // Check if the dataverse exists
+ Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
+ if (dv == null) {
+ throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc, dataverseName);
+ }
+
ds = metadataProvider.findDataset(dataverseName, datasetName);
if (ds == null) {
throw new CompilationException(ErrorCode.UNKNOWN_DATASET_IN_DATAVERSE, sourceLoc, datasetName,
dataverseName);
}
+ DatasetType datasetType = ds.getDatasetType();
+ validateIndexType(datasetType, indexType, isSecondaryPrimary, sourceLoc);
+
index = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
datasetName, indexName);
if (index != null) {
@@ -813,14 +951,11 @@
throw new CompilationException(ErrorCode.INDEX_EXISTS, sourceLoc, indexName);
}
}
+
// find keySourceIndicators for secondary primary index since the parser isn't aware of them
- if (isSecondaryPrimary && ds.getDatasetType() == DatasetType.INTERNAL) {
+ if (isSecondaryPrimary && datasetType == DatasetType.INTERNAL) {
keySourceIndicators = ((InternalDatasetDetails) ds.getDatasetDetails()).getKeySourceIndicator();
}
- // disable creating secondary primary index on an external dataset
- if (isSecondaryPrimary && ds.getDatasetType() == DatasetType.EXTERNAL) {
- throw new AsterixException(ErrorCode.CANNOT_CREATE_SEC_PRIMARY_IDX_ON_EXT_DATASET);
- }
// disable creating an index on meta fields (fields with source indicator == 1 are meta fields)
if (keySourceIndicators.stream().anyMatch(fieldSource -> fieldSource == 1) && !isSecondaryPrimary) {
throw new AsterixException(ErrorCode.COMPILATION_ERROR, sourceLoc,
@@ -865,9 +1000,8 @@
if (fieldExpr.second == null) {
fieldType = subType.getSubFieldType(fieldExpr.first.subList(i, fieldExpr.first.size()));
} else {
- if (!stmtCreateIndex.isEnforced() && stmtCreateIndex.getIndexType() != IndexType.BTREE) {
- throw new AsterixException(ErrorCode.INDEX_ILLEGAL_NON_ENFORCED_TYPED, sourceLoc,
- stmtCreateIndex.getIndexType());
+ if (!stmtCreateIndex.isEnforced() && indexType != IndexType.BTREE) {
+ throw new AsterixException(ErrorCode.INDEX_ILLEGAL_NON_ENFORCED_TYPED, sourceLoc, indexType);
}
if (stmtCreateIndex.isEnforced() && !fieldExpr.second.isUnknownable()) {
throw new AsterixException(ErrorCode.INDEX_ILLEGAL_ENFORCED_NON_OPTIONAL, sourceLoc,
@@ -921,10 +1055,10 @@
// Currently, we do not support this. Therefore, as a temporary solution, we
// print an
// error message and stop.
- if (stmtCreateIndex.getIndexType() == IndexType.SINGLE_PARTITION_WORD_INVIX
- || stmtCreateIndex.getIndexType() == IndexType.SINGLE_PARTITION_NGRAM_INVIX
- || stmtCreateIndex.getIndexType() == IndexType.LENGTH_PARTITIONED_WORD_INVIX
- || stmtCreateIndex.getIndexType() == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX) {
+ if (indexType == IndexType.SINGLE_PARTITION_WORD_INVIX
+ || indexType == IndexType.SINGLE_PARTITION_NGRAM_INVIX
+ || indexType == IndexType.LENGTH_PARTITIONED_WORD_INVIX
+ || indexType == IndexType.LENGTH_PARTITIONED_NGRAM_INVIX) {
List<List<String>> partitioningKeys = ds.getPrimaryKeys();
for (List<String> partitioningKey : partitioningKeys) {
IAType keyType = aRecordType.getSubFieldType(partitioningKey);
@@ -941,9 +1075,9 @@
}
}
- Index newIndex = new Index(dataverseName, datasetName, indexName, stmtCreateIndex.getIndexType(),
- indexFields, keySourceIndicators, indexFieldTypes, stmtCreateIndex.getGramLength(),
- overridesFieldTypes, stmtCreateIndex.isEnforced(), false, MetadataUtil.PENDING_ADD_OP);
+ Index newIndex = new Index(dataverseName, datasetName, indexName, indexType, indexFields,
+ keySourceIndicators, indexFieldTypes, stmtCreateIndex.getGramLength(), overridesFieldTypes,
+ stmtCreateIndex.isEnforced(), false, MetadataUtil.PENDING_ADD_OP);
doCreateIndex(hcc, metadataProvider, ds, newIndex, jobFlags, sourceLoc);
} finally {
metadataProvider.getLocks().unlock();
@@ -1194,6 +1328,14 @@
}
}
+ protected void validateIndexType(DatasetType datasetType, IndexType indexType, boolean isSecondaryPrimaryIndex,
+ SourceLocation sourceLoc) throws AlgebricksException {
+ // disable creating secondary primary index on an external dataset
+ if (datasetType == DatasetType.EXTERNAL && isSecondaryPrimaryIndex) {
+ throw new CompilationException(ErrorCode.CANNOT_CREATE_SEC_PRIMARY_IDX_ON_EXT_DATASET);
+ }
+ }
+
protected void validateIndexKeyFields(CreateIndexStatement stmtCreateIndex, List<Integer> keySourceIndicators,
ARecordType aRecordType, ARecordType metaRecordType, List<List<String>> indexFields,
List<IAType> indexFieldTypes) throws AlgebricksException {
@@ -1206,6 +1348,7 @@
SourceLocation sourceLoc = stmtCreateType.getSourceLocation();
String dataverseName = getActiveDataverse(stmtCreateType.getDataverseName());
String typeName = stmtCreateType.getIdent().getValue();
+ validateDatabaseObjectName(typeName, sourceLoc);
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
metadataProvider.setMetadataTxnContext(mdTxnCtx);
MetadataLockUtil.createTypeBegin(lockManager, metadataProvider.getLocks(), dataverseName,
@@ -1225,10 +1368,7 @@
throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
"Cannot redefine builtin type " + typeName + ".");
} else {
- Map<TypeSignature, IAType> typeMap = TypeTranslator.computeTypes(mdTxnCtx,
- stmtCreateType.getTypeDef(), stmtCreateType.getIdent().getValue(), dataverseName);
- TypeSignature typeSignature = new TypeSignature(dataverseName, typeName);
- IAType type = typeMap.get(typeSignature);
+ IAType type = translateType(dataverseName, typeName, stmtCreateType.getTypeDef(), mdTxnCtx);
MetadataManager.INSTANCE.addDatatype(mdTxnCtx, new Datatype(dataverseName, typeName, type, false));
}
}
@@ -1241,6 +1381,13 @@
}
}
+ private IAType translateType(String dataverseName, String typeName, TypeExpression typeDef,
+ MetadataTransactionContext mdTxnCtx) throws AlgebricksException {
+ Map<TypeSignature, IAType> typeMap = TypeTranslator.computeTypes(mdTxnCtx, typeDef, typeName, dataverseName);
+ TypeSignature typeSignature = new TypeSignature(dataverseName, typeName);
+ return typeMap.get(typeSignature);
+ }
+
protected void handleDataverseDropStatement(MetadataProvider metadataProvider, Statement stmt,
IHyracksClientConnection hcc, IRequestParameters requestParameters) throws Exception {
DataverseDropStatement stmtDelete = (DataverseDropStatement) stmt;
@@ -1439,6 +1586,20 @@
metadataProvider.setMetadataTxnContext(mdTxnCtx.getValue());
List<JobSpecification> jobsToExecute = new ArrayList<>();
try {
+ // Check if the dataverse exists
+ Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx.getValue(), dataverseName);
+ if (dv == null) {
+ if (ifExists) {
+ if (warningCollector.shouldWarn()) {
+ warningCollector
+ .warn(WarningUtil.forAsterix(sourceLoc, ErrorCode.UNKNOWN_DATAVERSE, dataverseName));
+ }
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx.getValue());
+ return;
+ } else {
+ throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc, dataverseName);
+ }
+ }
Dataset ds = metadataProvider.findDataset(dataverseName, datasetName);
if (ds == null) {
if (ifExists) {
@@ -1450,8 +1611,41 @@
}
}
validateDatasetState(metadataProvider, ds, sourceLoc);
+
+ // prepare to drop item and meta types if they were created as inline types
+ String itemTypeDataverseName = ds.getItemTypeDataverseName();
+ String itemTypeName = ds.getItemTypeName();
+ boolean isInlineItemType = DatasetUtil.isInlineTypeName(ds, itemTypeDataverseName, itemTypeName);
+ if (isInlineItemType) {
+ MetadataLockUtil.dropTypeBegin(lockManager, metadataProvider.getLocks(), itemTypeDataverseName,
+ itemTypeDataverseName + '.' + itemTypeName);
+ }
+ String metaTypeDataverseName = ds.getMetaItemTypeDataverseName();
+ String metaTypeName = ds.getMetaItemTypeName();
+ boolean isInlineMetaType =
+ metaTypeName != null && DatasetUtil.isInlineTypeName(ds, metaTypeDataverseName, metaTypeName);
+ if (isInlineMetaType) {
+ MetadataLockUtil.dropTypeBegin(lockManager, metadataProvider.getLocks(), metaTypeDataverseName,
+ metaTypeDataverseName + '.' + metaTypeName);
+ }
+ Datatype inlineItemType = isInlineItemType
+ ? MetadataManager.INSTANCE.getDatatype(mdTxnCtx.getValue(), itemTypeDataverseName, itemTypeName)
+ : null;
+ Datatype inlineMetaType = isInlineMetaType
+ ? MetadataManager.INSTANCE.getDatatype(mdTxnCtx.getValue(), metaTypeDataverseName, metaTypeName)
+ : null;
+
ds.drop(metadataProvider, mdTxnCtx, jobsToExecute, bActiveTxn, progress, hcc, dropCorrespondingNodeGroup,
sourceLoc);
+
+ // drop inline item and meta types
+ if (isInlineItemType && inlineItemType.getIsAnonymous()) {
+ MetadataManager.INSTANCE.dropDatatype(mdTxnCtx.getValue(), itemTypeDataverseName, itemTypeName);
+ }
+ if (isInlineMetaType && inlineMetaType.getIsAnonymous()) {
+ MetadataManager.INSTANCE.dropDatatype(mdTxnCtx.getValue(), metaTypeDataverseName, metaTypeName);
+ }
+
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx.getValue());
} catch (Exception e) {
if (bActiveTxn.booleanValue()) {
@@ -1681,6 +1875,21 @@
MetadataLockUtil.dropTypeBegin(lockManager, metadataProvider.getLocks(), dataverseName,
dataverseName + "." + typeName);
try {
+ // Check if the dataverse exists
+ Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
+ if (dv == null) {
+ if (stmtTypeDrop.getIfExists()) {
+ if (warningCollector.shouldWarn()) {
+ warningCollector
+ .warn(WarningUtil.forAsterix(sourceLoc, ErrorCode.UNKNOWN_DATAVERSE, dataverseName));
+ }
+ MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+ return;
+ } else {
+ throw new CompilationException(ErrorCode.UNKNOWN_DATAVERSE, sourceLoc, dataverseName);
+ }
+ }
+
Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataverseName, typeName);
if (dt == null) {
if (!stmtTypeDrop.getIfExists()) {
@@ -1728,6 +1937,7 @@
CreateFunctionStatement cfs = (CreateFunctionStatement) stmt;
SourceLocation sourceLoc = cfs.getSourceLocation();
FunctionSignature signature = cfs.getFunctionSignature();
+ validateDatabaseObjectName(signature.getName(), sourceLoc);
String dataverse = getActiveDataverseName(signature.getNamespace());
signature.setNamespace(dataverse);
@@ -1844,9 +2054,12 @@
MetadataLockUtil.modifyDatasetBegin(lockManager, metadataProvider.getLocks(), dataverseName,
dataverseName + "." + datasetName);
try {
+ Map<String, String> properties = loadStmt.getProperties();
+ ExternalDataUtils.normalize(properties);
+ ExternalDataUtils.validate(properties);
CompiledLoadFromFileStatement cls =
new CompiledLoadFromFileStatement(dataverseName, loadStmt.getDatasetName().getValue(),
- loadStmt.getAdapter(), loadStmt.getProperties(), loadStmt.dataIsAlreadySorted());
+ loadStmt.getAdapter(), properties, loadStmt.dataIsAlreadySorted());
cls.setSourceLocation(stmt.getSourceLocation());
JobSpecification spec = apiFramework.compileQuery(hcc, metadataProvider, null, 0, null, sessionOutput, cls,
null, responsePrinter, warningCollector);
@@ -2028,6 +2241,7 @@
SourceLocation sourceLoc = cfs.getSourceLocation();
String dataverseName = getActiveDataverse(cfs.getDataverseName());
String feedName = cfs.getFeedName().getValue();
+ validateDatabaseObjectName(feedName, sourceLoc);
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
metadataProvider.setMetadataTxnContext(mdTxnCtx);
MetadataLockUtil.createFeedBegin(lockManager, metadataProvider.getLocks(), dataverseName,
@@ -2044,8 +2258,11 @@
"A feed with this name " + feedName + " already exists.");
}
}
- feed = new Feed(dataverseName, feedName, cfs.getConfiguration());
- FeedMetadataUtil.validateFeed(feed, mdTxnCtx, appCtx);
+ Map<String, String> configuration = cfs.getConfiguration();
+ ExternalDataUtils.normalize(configuration);
+ ExternalDataUtils.validate(configuration);
+ feed = new Feed(dataverseName, feedName, configuration);
+ FeedMetadataUtil.validateFeed(feed, mdTxnCtx, appCtx, warningCollector);
MetadataManager.INSTANCE.addFeed(metadataProvider.getMetadataTxnContext(), feed);
MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
} catch (Exception e) {
@@ -2066,6 +2283,7 @@
SourceLocation sourceLoc = cfps.getSourceLocation();
dataverse = getActiveDataverse(null);
policy = cfps.getPolicyName();
+ validateDatabaseObjectName(policy, sourceLoc);
MetadataLockUtil.createFeedPolicyBegin(lockManager, metadataProvider.getLocks(), dataverse,
dataverse + "." + policy);
try {
@@ -2662,6 +2880,7 @@
NodegroupDecl stmtCreateNodegroup = (NodegroupDecl) stmt;
SourceLocation sourceLoc = stmtCreateNodegroup.getSourceLocation();
String ngName = stmtCreateNodegroup.getNodegroupName().getValue();
+ validateDatabaseObjectName(ngName, sourceLoc);
MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
metadataProvider.setMetadataTxnContext(mdTxnCtx);
@@ -3051,4 +3270,41 @@
throw new RuntimeDataException(ErrorCode.REQUEST_CANCELLED, clientRequest.getId());
}
}
+
+ protected void validateExternalDatasetProperties(ExternalDetailsDecl externalDetails,
+ Map<String, String> properties, SourceLocation srcLoc, MetadataTransactionContext mdTxnCtx)
+ throws AlgebricksException, HyracksDataException {
+ // Validate adapter specific properties
+ String adapter = externalDetails.getAdapter();
+ Map<String, String> details = new HashMap<>(properties);
+ details.put(ExternalDataConstants.KEY_EXTERNAL_SOURCE_TYPE, adapter);
+ validateAdapterSpecificProperties(details, srcLoc);
+ }
+
+ /**
+ * Ensures that the external source container is present
+ *
+ * @param configuration external source properties
+ */
+ protected void validateAdapterSpecificProperties(Map<String, String> configuration, SourceLocation srcLoc)
+ throws CompilationException {
+ ExternalDataUtils.validateAdapterSpecificProperties(configuration, srcLoc, warningCollector);
+ }
+
+ private static void validateDataverseName(String dataverseName, SourceLocation sourceLoc)
+ throws CompilationException {
+ validateDatabaseObjectName(dataverseName, sourceLoc);
+ if (dataverseName.contains(".")) {
+ throw new CompilationException(ErrorCode.INVALID_DATABASE_OBJECT_NAME, sourceLoc, dataverseName);
+ }
+ }
+
+ public static void validateDatabaseObjectName(String name, SourceLocation sourceLoc) throws CompilationException {
+ if (name == null || name.isEmpty()) {
+ throw new CompilationException(ErrorCode.INVALID_DATABASE_OBJECT_NAME, sourceLoc, "<empty>");
+ }
+ if (Character.isWhitespace(name.codePointAt(0))) {
+ throw new CompilationException(ErrorCode.INVALID_DATABASE_OBJECT_NAME, sourceLoc, name);
+ }
+ }
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/RequestParameters.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/RequestParameters.java
index 90602e7..5ebc9ba 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/RequestParameters.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/RequestParameters.java
@@ -31,6 +31,7 @@
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.translator.IRequestParameters;
import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.asterix.translator.IStatementExecutor.StatementProperties;
import org.apache.asterix.translator.IStatementExecutor.Stats;
import org.apache.asterix.translator.ResultProperties;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -49,6 +50,7 @@
private final IResultSet resultSet;
private final ResultProperties resultProperties;
private final Stats stats;
+ private final StatementProperties statementProperties;
private final Map<String, String> optionalParameters;
private final IStatementExecutor.ResultMetadata outMetadata;
private final String clientContextId;
@@ -58,22 +60,24 @@
private final String statement;
public RequestParameters(IRequestReference requestReference, String statement, IResultSet resultSet,
- ResultProperties resultProperties, Stats stats, IStatementExecutor.ResultMetadata outMetadata,
- String clientContextId, Map<String, String> optionalParameters, Map<String, IAObject> statementParameters,
- boolean multiStatement) {
- this(requestReference, statement, resultSet, resultProperties, stats, outMetadata, clientContextId,
- optionalParameters, statementParameters, multiStatement, NO_CATEGORY_RESTRICTION_MASK);
+ ResultProperties resultProperties, Stats stats, StatementProperties statementProperties,
+ IStatementExecutor.ResultMetadata outMetadata, String clientContextId,
+ Map<String, String> optionalParameters, Map<String, IAObject> statementParameters, boolean multiStatement) {
+ this(requestReference, statement, resultSet, resultProperties, stats, statementProperties, outMetadata,
+ clientContextId, optionalParameters, statementParameters, multiStatement, NO_CATEGORY_RESTRICTION_MASK);
}
public RequestParameters(IRequestReference requestReference, String statement, IResultSet resultSet,
- ResultProperties resultProperties, Stats stats, IStatementExecutor.ResultMetadata outMetadata,
- String clientContextId, Map<String, String> optionalParameters, Map<String, IAObject> statementParameters,
- boolean multiStatement, int statementCategoryRestrictionMask) {
+ ResultProperties resultProperties, Stats stats, StatementProperties statementProperties,
+ IStatementExecutor.ResultMetadata outMetadata, String clientContextId,
+ Map<String, String> optionalParameters, Map<String, IAObject> statementParameters, boolean multiStatement,
+ int statementCategoryRestrictionMask) {
this.requestReference = requestReference;
this.statement = statement;
this.resultSet = resultSet;
this.resultProperties = resultProperties;
this.stats = stats;
+ this.statementProperties = statementProperties;
this.outMetadata = outMetadata;
this.clientContextId = clientContextId;
this.optionalParameters = optionalParameters;
@@ -93,11 +97,16 @@
}
@Override
- public IStatementExecutor.Stats getStats() {
+ public Stats getStats() {
return stats;
}
@Override
+ public StatementProperties getStatementProperties() {
+ return statementProperties;
+ }
+
+ @Override
public Map<String, String> getOptionalParameters() {
return optionalParameters;
}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
index 26c092f..0f0620c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
@@ -69,9 +69,11 @@
import org.apache.asterix.common.config.ReplicationProperties;
import org.apache.asterix.common.context.IStorageComponentProvider;
import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.external.IAdapterFactoryService;
import org.apache.asterix.common.library.ILibraryManager;
import org.apache.asterix.common.replication.INcLifecycleCoordinator;
import org.apache.asterix.common.utils.Servlets;
+import org.apache.asterix.external.adapter.factory.AdapterFactoryService;
import org.apache.asterix.external.library.ExternalLibraryManager;
import org.apache.asterix.file.StorageComponentProvider;
import org.apache.asterix.messaging.CCMessageBroker;
@@ -84,6 +86,7 @@
import org.apache.asterix.translator.IStatementExecutorFactory;
import org.apache.asterix.translator.Receptionist;
import org.apache.asterix.util.MetadataBuiltinFunctions;
+import org.apache.asterix.utils.RedactionUtil;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.client.IHyracksClientConnection;
@@ -102,6 +105,7 @@
import org.apache.hyracks.http.server.HttpServerConfigBuilder;
import org.apache.hyracks.http.server.WebManager;
import org.apache.hyracks.ipc.impl.HyracksConnection;
+import org.apache.hyracks.util.LogRedactionUtil;
import org.apache.hyracks.util.LoggingConfigUtil;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
@@ -154,7 +158,7 @@
ccExtensionManager = new CCExtensionManager(new ArrayList<>(getExtensions()));
IGlobalRecoveryManager globalRecoveryManager = createGlobalRecoveryManager();
appCtx = createApplicationContext(libraryManager, globalRecoveryManager, lifecycleCoordinator,
- () -> new Receptionist("CC"), ConfigValidator::new, ccExtensionManager);
+ () -> new Receptionist("CC"), ConfigValidator::new, ccExtensionManager, new AdapterFactoryService());
final CCConfig ccConfig = controllerService.getCCConfig();
if (System.getProperty("java.rmi.server.hostname") == null) {
System.setProperty("java.rmi.server.hostname", ccConfig.getClusterPublicAddress());
@@ -182,10 +186,12 @@
protected ICcApplicationContext createApplicationContext(ILibraryManager libraryManager,
IGlobalRecoveryManager globalRecoveryManager, INcLifecycleCoordinator lifecycleCoordinator,
IReceptionistFactory receptionistFactory, IConfigValidatorFactory configValidatorFactory,
- CCExtensionManager ccExtensionManager) throws AlgebricksException, IOException {
+ CCExtensionManager ccExtensionManager, IAdapterFactoryService adapterFactoryService)
+ throws AlgebricksException, IOException {
return new CcApplicationContext(ccServiceCtx, getHcc(), libraryManager, () -> MetadataManager.INSTANCE,
globalRecoveryManager, lifecycleCoordinator, new ActiveNotificationHandler(), componentProvider,
- new MetadataLockManager(), receptionistFactory, configValidatorFactory, ccExtensionManager);
+ new MetadataLockManager(), receptionistFactory, configValidatorFactory, ccExtensionManager,
+ adapterFactoryService);
}
protected IGlobalRecoveryManager createGlobalRecoveryManager() throws Exception {
@@ -200,6 +206,7 @@
public void configureLoggingLevel(Level level) {
super.configureLoggingLevel(level);
LoggingConfigUtil.defaultIfMissing(GlobalConfig.ASTERIX_LOGGER_NAME, level);
+ LogRedactionUtil.setRedactor(RedactionUtil.LOG_REDACTOR);
}
protected List<AsterixExtension> getExtensions() throws Exception {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
index 2e5c09c..1036fb2 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
@@ -72,6 +72,7 @@
import org.apache.asterix.transaction.management.resource.PersistentLocalResourceRepository;
import org.apache.asterix.translator.Receptionist;
import org.apache.asterix.util.MetadataBuiltinFunctions;
+import org.apache.asterix.utils.RedactionUtil;
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.client.NodeStatus;
import org.apache.hyracks.api.config.IConfigManager;
@@ -86,6 +87,7 @@
import org.apache.hyracks.http.server.HttpServerConfig;
import org.apache.hyracks.http.server.HttpServerConfigBuilder;
import org.apache.hyracks.http.server.WebManager;
+import org.apache.hyracks.util.LogRedactionUtil;
import org.apache.hyracks.util.LoggingConfigUtil;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
@@ -184,6 +186,7 @@
public void configureLoggingLevel(Level level) {
super.configureLoggingLevel(level);
LoggingConfigUtil.defaultIfMissing(GlobalConfig.ASTERIX_LOGGER_NAME, level);
+ LogRedactionUtil.setRedactor(RedactionUtil.LOG_REDACTOR);
}
protected void configureServers() throws Exception {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
index b74f4c6..fc64d99 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
@@ -42,7 +42,7 @@
import org.apache.asterix.common.transactions.TxnId;
import org.apache.asterix.common.utils.StoragePathUtil;
import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
-import org.apache.asterix.external.api.IAdapterFactory;
+import org.apache.asterix.external.api.ITypedAdapterFactory;
import org.apache.asterix.external.feed.management.FeedConnectionId;
import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
import org.apache.asterix.external.feed.watch.FeedActivityDetails;
@@ -135,14 +135,14 @@
private FeedOperations() {
}
- private static Pair<JobSpecification, IAdapterFactory> buildFeedIntakeJobSpec(Feed feed,
+ private static Pair<JobSpecification, ITypedAdapterFactory> buildFeedIntakeJobSpec(Feed feed,
MetadataProvider metadataProvider, FeedPolicyAccessor policyAccessor) throws Exception {
JobSpecification spec = RuntimeUtils.createJobSpecification(metadataProvider.getApplicationContext());
spec.setFrameSize(metadataProvider.getApplicationContext().getCompilerProperties().getFrameSize());
- IAdapterFactory adapterFactory;
+ ITypedAdapterFactory adapterFactory;
IOperatorDescriptor feedIngestor;
AlgebricksPartitionConstraint ingesterPc;
- Triple<IOperatorDescriptor, AlgebricksPartitionConstraint, IAdapterFactory> t =
+ Triple<IOperatorDescriptor, AlgebricksPartitionConstraint, ITypedAdapterFactory> t =
metadataProvider.buildFeedIntakeRuntime(spec, feed, policyAccessor);
feedIngestor = t.first;
ingesterPc = t.second;
@@ -447,13 +447,13 @@
MetadataProvider metadataProvider, Feed feed, List<FeedConnection> feedConnections,
IStatementExecutor statementExecutor, IHyracksClientConnection hcc) throws Exception {
FeedPolicyAccessor fpa = new FeedPolicyAccessor(new HashMap<>());
- Pair<JobSpecification, IAdapterFactory> intakeInfo = buildFeedIntakeJobSpec(feed, metadataProvider, fpa);
+ Pair<JobSpecification, ITypedAdapterFactory> intakeInfo = buildFeedIntakeJobSpec(feed, metadataProvider, fpa);
List<JobSpecification> jobsList = new ArrayList<>();
// TODO: Figure out a better way to handle insert/upsert per conn instead of per feed
Boolean insertFeed = ExternalDataUtils.isInsertFeed(feed.getConfiguration());
// Construct the ingestion Job
JobSpecification intakeJob = intakeInfo.getLeft();
- IAdapterFactory ingestionAdaptorFactory = intakeInfo.getRight();
+ ITypedAdapterFactory ingestionAdaptorFactory = intakeInfo.getRight();
String[] ingestionLocations = ingestionAdaptorFactory.getPartitionConstraint().getLocations();
// Add metadata configs
metadataProvider.getConfig().put(FunctionUtil.IMPORT_PRIVATE_FUNCTIONS, Boolean.TRUE.toString());
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
index 483987c..25076c2 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RebalanceUtil.java
@@ -19,6 +19,7 @@
package org.apache.asterix.utils;
import static org.apache.asterix.app.translator.QueryTranslator.abort;
+import static org.apache.asterix.common.config.DatasetConfig.DatasetType;
import static org.apache.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor.DropOption;
import java.rmi.RemoteException;
@@ -92,7 +93,7 @@
*/
public static void rebalance(String dataverseName, String datasetName, Set<String> targetNcNames,
MetadataProvider metadataProvider, IHyracksClientConnection hcc,
- IDatasetRebalanceCallback datasetRebalanceCallback) throws Exception {
+ IDatasetRebalanceCallback datasetRebalanceCallback, boolean forceRebalance) throws Exception {
Dataset sourceDataset;
Dataset targetDataset;
// Executes the first Metadata transaction.
@@ -111,8 +112,7 @@
Set<String> sourceNodes = new HashSet<>(metadataProvider.findNodes(sourceDataset.getNodeGroupName()));
- // The the source nodes are identical to the target nodes.
- if (sourceNodes.equals(targetNcNames)) {
+ if (!forceRebalance && sourceNodes.equals(targetNcNames)) {
return;
}
@@ -128,7 +128,9 @@
sourceDataset.getDatasetName(), sourceDataset.getNodeGroupName(), sourceNodes,
targetDataset.getNodeGroupName(), targetNcNames);
// Rebalances the source dataset into the target dataset.
- rebalance(sourceDataset, targetDataset, metadataProvider, hcc, datasetRebalanceCallback);
+ if (sourceDataset.getDatasetType() != DatasetType.EXTERNAL) {
+ rebalance(sourceDataset, targetDataset, metadataProvider, hcc, datasetRebalanceCallback);
+ }
} else {
targetDataset = null;
// if this the last NC in the cluster, just drop the dataset
@@ -351,6 +353,9 @@
// Drops dataset files of a given dataset.
private static void dropDatasetFiles(Dataset dataset, MetadataProvider metadataProvider,
IHyracksClientConnection hcc) throws Exception {
+ if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+ return;
+ }
List<JobSpecification> jobs = new ArrayList<>();
List<Index> indexes = metadataProvider.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
for (Index index : indexes) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java
new file mode 100644
index 0000000..156b78a
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/RedactionUtil.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.utils;
+
+import static java.util.regex.Pattern.CASE_INSENSITIVE;
+import static java.util.regex.Pattern.DOTALL;
+import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.SECRET_ACCESS_KEY_FIELD_NAME;
+
+import java.util.regex.Pattern;
+
+import org.apache.hyracks.util.ILogRedactor;
+
+public class RedactionUtil {
+ private RedactionUtil() {
+ throw new AssertionError("do not instantiate");
+ }
+
+ private static final Pattern STATEMENT_PATTERN =
+ Pattern.compile("(" + SECRET_ACCESS_KEY_FIELD_NAME + ").*", CASE_INSENSITIVE | DOTALL);
+ private static final String STATEMENT_REPLACEMENT = "$1...<redacted sensitive data>";
+
+ public static final ILogRedactor LOG_REDACTOR = new ILogRedactor() {
+ @Override
+ public String userData(String text) {
+ return text;
+ }
+
+ @Override
+ public String statement(String text) {
+ return STATEMENT_PATTERN.matcher(text).replaceFirst(STATEMENT_REPLACEMENT);
+ }
+
+ @Override
+ public String unredactUserData(String text) {
+ return text;
+ }
+ };
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java
index 348b947..50e1155 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java
@@ -70,8 +70,8 @@
verify(mockResponse, times(1)).setStatus(HttpResponseStatus.NOT_FOUND);
final RequestReference requestReference = RequestReference.of("1", "node1", System.currentTimeMillis());
- RequestParameters requestParameters =
- new RequestParameters(requestReference, "select 1", null, null, null, null, "1", null, null, true);
+ RequestParameters requestParameters = new RequestParameters(requestReference, "select 1", null, null, null,
+ null, null, "1", null, null, true);
ClientRequest request = new ClientRequest(requestParameters);
request.setJobId(new JobId(1));
request.markCancellable();
@@ -87,8 +87,8 @@
// Tests the case that the job cancellation hit some exception from Hyracks.
final RequestReference requestReference2 = RequestReference.of("2", "node1", System.currentTimeMillis());
- requestParameters =
- new RequestParameters(requestReference2, "select 1", null, null, null, null, "2", null, null, true);
+ requestParameters = new RequestParameters(requestReference2, "select 1", null, null, null, null, null, "2",
+ null, null, true);
ClientRequest request2 = new ClientRequest(requestParameters);
request2.setJobId(new JobId(2));
request2.markCancellable();
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/common/TestDataUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/common/TestDataUtil.java
index d453824..f15b88f 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/common/TestDataUtil.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/common/TestDataUtil.java
@@ -167,7 +167,7 @@
lockManager.acquireDatasetExclusiveModificationLock(metadataProvider.getLocks(),
dataverseName + '.' + datasetName);
RebalanceUtil.rebalance(dataverseName, datasetName, new LinkedHashSet<>(Arrays.asList(targetNodes)),
- metadataProvider, ccAppCtx.getHcc(), NoOpDatasetRebalanceCallback.INSTANCE);
+ metadataProvider, ccAppCtx.getHcc(), NoOpDatasetRebalanceCallback.INSTANCE, false);
} finally {
activeNotificationHandler.resume(metadataProvider);
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
index 64520a4..c43f951 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveEventsListenerTest.java
@@ -14,10 +14,12 @@
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
- * under the License.
+ * under the License
*/
package org.apache.asterix.test.active;
+import static org.apache.hyracks.api.exceptions.HyracksException.UNKNOWN;
+
import java.util.ArrayList;
import java.util.Collections;
import java.util.EnumSet;
@@ -57,6 +59,7 @@
import org.apache.hyracks.api.client.IHyracksClientConnection;
import org.apache.hyracks.api.control.CcId;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IError;
import org.apache.hyracks.api.job.JobIdFactory;
import org.apache.hyracks.api.job.JobStatus;
import org.apache.hyracks.control.cc.ClusterControllerService;
@@ -181,7 +184,7 @@
listener.onStart(Behavior.FAIL_COMPILE);
Action action = users[0].startActivity(listener);
action.sync();
- assertFailure(action, 0);
+ assertUnknownFailure(action);
Assert.assertEquals(ActivityState.STOPPED, listener.getState());
}
@@ -191,7 +194,7 @@
listener.onStart(Behavior.FAIL_RUNTIME);
Action action = users[0].startActivity(listener);
action.sync();
- assertFailure(action, 0);
+ assertUnknownFailure(action);
Assert.assertEquals(ActivityState.STOPPED, listener.getState());
}
@@ -211,7 +214,7 @@
listener.onStart(Behavior.FAIL_START_TIMEOUT_STUCK);
Action action = users[0].startActivity(listener);
action.sync();
- assertFailure(action, 0);
+ assertUnknownFailure(action);
Assert.assertEquals(ActivityState.STOPPED, listener.getState());
}
@@ -1554,17 +1557,28 @@
assertSuccess(addDataset);
}
- private void assertFailure(Action action, int errorCode) throws Exception {
+ private void assertFailure(Action action, IError errorCode) throws Exception {
HyracksDataException exception = action.getFailure();
try {
Assert.assertTrue(action.hasFailed());
Assert.assertNotNull(exception);
- Assert.assertEquals(errorCode, exception.getErrorCode());
+ Assert.assertTrue(exception.matches(errorCode));
} catch (Exception e) {
throw new Exception("Expected failure: " + errorCode + ". Found failure: " + exception);
}
}
+ private void assertUnknownFailure(Action action) throws Exception {
+ HyracksDataException exception = action.getFailure();
+ try {
+ Assert.assertTrue(action.hasFailed());
+ Assert.assertNotNull(exception);
+ Assert.assertEquals(UNKNOWN, exception.getErrorCode());
+ } catch (Exception e) {
+ throw new Exception("Expected failure: " + UNKNOWN + ". Found failure: " + exception);
+ }
+ }
+
private void assertSuccess(Action action) throws Exception {
if (action.hasFailed()) {
System.err.println("Action failed while it was expected to succeed");
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
index 80dde8a..df4c9e0 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
@@ -19,6 +19,8 @@
package org.apache.asterix.test.active;
+import static org.apache.asterix.common.exceptions.ErrorCode.ACTIVE_ENTITY_NOT_RUNNING;
+
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
@@ -60,8 +62,8 @@
public class ActiveStatsTest {
protected boolean cleanUp = true;
- private static String EXPECTED_STATS = "\"Mock stats\"";
- private static String CONF_PATH = "src/main/resources/cc.conf";
+ private static final String EXPECTED_STATS = "\"Mock stats\"";
+ private static final String CONF_PATH = "src/main/resources/cc.conf";
@Before
public void setUp() throws Exception {
@@ -116,7 +118,13 @@
Assert.assertTrue(requestedStats.contains("N/A"));
// Update stats of not-started job
- eventsListener.refreshStats(1000);
+ try {
+ eventsListener.refreshStats(1000);
+ Assert.fail("expected exception on refresh stats on not-started job");
+ } catch (HyracksDataException e) {
+ Assert.assertTrue("incorrect exception thrown (expected: ACTIVE_ENTITY_NOT_RUNNING, was: " + e,
+ e.matches(ACTIVE_ENTITY_NOT_RUNNING));
+ }
requestedStats = eventsListener.getStats();
Assert.assertTrue(requestedStats.contains("N/A"));
WaitForStateSubscriber startingSubscriber =
@@ -127,8 +135,12 @@
startingSubscriber.sync();
activeJobNotificationHandler.notifyJobCreation(jobId, jobSpec);
activeJobNotificationHandler.notifyJobStart(jobId);
- eventsListener.refreshStats(1000);
- requestedStats = eventsListener.getStats();
+ try {
+ eventsListener.refreshStats(1000);
+ } catch (HyracksDataException e) {
+ Assert.assertTrue("incorrect exception thrown (expected: ACTIVE_ENTITY_NOT_RUNNING, was: " + e,
+ e.matches(ACTIVE_ENTITY_NOT_RUNNING));
+ }
Assert.assertTrue(requestedStats.contains("N/A"));
// Fake partition message and notify eventListener
ActivePartitionMessage partitionMessage =
@@ -159,7 +171,7 @@
expectedException = e;
}
Assert.assertNotNull(expectedException);
- Assert.assertEquals(ErrorCode.ACTIVE_MANAGER_INVALID_RUNTIME, expectedException.getErrorCode());
+ Assert.assertTrue(expectedException.matches(ErrorCode.ACTIVE_MANAGER_INVALID_RUNTIME));
}
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/CancellationTestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/CancellationTestExecutor.java
index 36a06c0..d4f92fb 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/CancellationTestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/CancellationTestExecutor.java
@@ -23,6 +23,7 @@
import java.net.URI;
import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
+import java.util.BitSet;
import java.util.Iterator;
import java.util.List;
import java.util.UUID;
@@ -129,8 +130,7 @@
}
@Override
- protected void ensureWarnings(int actualWarnCount, int expectedWarnCount, TestCase.CompilationUnit cUnit)
- throws Exception {
+ protected void ensureWarnings(BitSet expectedWarnings, TestCase.CompilationUnit cUnit) throws Exception {
// skip checking warnings as currently cancelled queries with warnings might not run successfully at all
}
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/IPollTask.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/IPollTask.java
index ab90244..a1ed12b 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/IPollTask.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/IPollTask.java
@@ -19,6 +19,7 @@
package org.apache.asterix.test.common;
import java.io.File;
+import java.util.BitSet;
import java.util.List;
import java.util.Map;
@@ -43,11 +44,11 @@
* @param expectedResultFileCtxs
* @param testFile
* @param actualPath
- * @param actualWarnCount
+ * @param expectedWarnings
*/
void execute(TestCaseContext testCaseCtx, TestFileContext ctx, Map<String, Object> variableCtx, String statement,
boolean isDmlRecoveryTest, ProcessBuilder pb, CompilationUnit cUnit, MutableInt queryCount,
- List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath, MutableInt actualWarnCount)
+ List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath, BitSet expectedWarnings)
throws Exception;
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java
index 5e53d54..17ba250 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java
@@ -108,6 +108,10 @@
return extract(resultStream, EnumSet.of(ResultField.PLANS), resultCharset).getResult();
}
+ public static InputStream extractStatus(InputStream resultStream, Charset resultCharset) throws Exception {
+ return extract(resultStream, EnumSet.of(ResultField.STATUS), resultCharset).getResult();
+ }
+
public static String extractHandle(InputStream resultStream, Charset responseCharset) throws Exception {
String result = IOUtils.toString(resultStream, responseCharset);
ObjectNode resultJson = OBJECT_MAPPER.readValue(result, ObjectNode.class);
@@ -140,16 +144,18 @@
return extract(resultStream, resultFields, resultCharset, "", "", "");
}
- throw new AsterixException("Unkown output format for result of test query");
+ throw new AsterixException("Unknown output format for result of test query");
}
private static ExtractedResult extract(InputStream resultStream, EnumSet<ResultField> resultFields,
Charset resultCharset, String openMarker, String separator, String closeMarker) throws Exception {
ExtractedResult extractedResult = new ExtractedResult();
final String resultStr = IOUtils.toString(resultStream, resultCharset);
+
+ LOGGER.debug("+++++++\n" + resultStr + "\n+++++++\n");
+
final ObjectNode result = OBJECT_MAPPER.readValue(resultStr, ObjectNode.class);
- LOGGER.debug("+++++++\n" + result + "\n+++++++\n");
// if we have errors field in the results, we will always return it
checkForErrors(result);
final StringBuilder resultBuilder = new StringBuilder();
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
index a10dc54..e586878 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
@@ -19,6 +19,7 @@
package org.apache.asterix.test.common;
import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hyracks.util.NetworkUtil.toHostPort;
import static org.apache.hyracks.util.file.FileUtil.canonicalize;
import java.io.BufferedReader;
@@ -45,6 +46,7 @@
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.BitSet;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
@@ -53,6 +55,7 @@
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
+import java.util.OptionalInt;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.ExecutionException;
@@ -65,6 +68,8 @@
import java.util.function.Predicate;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
+import java.util.regex.PatternSyntaxException;
+import java.util.stream.IntStream;
import java.util.stream.Stream;
import org.apache.asterix.api.http.server.QueryServiceRequestParameters;
@@ -151,6 +156,7 @@
Pattern.compile("maxresultreads=(\\d+)(\\D|$)", Pattern.MULTILINE);
private static final Pattern HTTP_REQUEST_TYPE = Pattern.compile("requesttype=(.*)", Pattern.MULTILINE);
private static final Pattern EXTRACT_RESULT_TYPE = Pattern.compile("extractresult=(.*)", Pattern.MULTILINE);
+ private static final Pattern EXTRACT_STATUS_PATTERN = Pattern.compile("extractstatus", Pattern.MULTILINE);
private static final String NC_ENDPOINT_PREFIX = "nc:";
public static final int TRUNCATE_THRESHOLD = 16384;
public static final Set<String> NON_CANCELLABLE =
@@ -159,9 +165,9 @@
private static final ContentType TEXT_PLAIN_UTF8 = ContentType.create(HttpUtil.ContentType.APPLICATION_JSON, UTF_8);
private final IPollTask plainExecutor = (testCaseCtx, ctx, variableCtx, statement, isDmlRecoveryTest, pb, cUnit,
- queryCount, expectedResultFileCtxs, testFile, actualPath, actualWarnCount) -> executeTestFile(testCaseCtx,
+ queryCount, expectedResultFileCtxs, testFile, actualPath, expectedWarnings) -> executeTestFile(testCaseCtx,
ctx, variableCtx, statement, isDmlRecoveryTest, pb, cUnit, queryCount, expectedResultFileCtxs,
- testFile, actualPath, actualWarnCount);
+ testFile, actualPath, expectedWarnings);
public static final String DELIVERY_ASYNC = "async";
public static final String DELIVERY_DEFERRED = "deferred";
@@ -947,18 +953,21 @@
public void executeTestFile(TestCaseContext testCaseCtx, TestFileContext ctx, Map<String, Object> variableCtx,
String statement, boolean isDmlRecoveryTest, ProcessBuilder pb, CompilationUnit cUnit,
MutableInt queryCount, List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath,
- MutableInt actualWarnCount) throws Exception {
+ BitSet expectedWarnings) throws Exception {
InputStream resultStream;
File qbcFile;
boolean failed = false;
File expectedResultFile;
switch (ctx.getType()) {
case "ddl":
+ ExtractedResult ddlExtractedResult;
if (ctx.getFile().getName().endsWith("aql")) {
- executeAqlUpdateOrDdl(statement, OutputFormat.CLEAN_JSON);
+ ddlExtractedResult = executeAqlUpdateOrDdl(statement, OutputFormat.CLEAN_JSON);
} else {
- executeSqlppUpdateOrDdl(statement, OutputFormat.CLEAN_JSON);
+ ddlExtractedResult = executeSqlppUpdateOrDdl(statement, OutputFormat.CLEAN_JSON);
}
+
+ validateWarning(ddlExtractedResult, testCaseCtx, cUnit, testFile, expectedWarnings);
break;
case "update":
// isDmlRecoveryTest: set IP address
@@ -975,11 +984,11 @@
case "pollquery":
poll(testCaseCtx, ctx, variableCtx, statement, isDmlRecoveryTest, pb, cUnit, queryCount,
expectedResultFileCtxs, testFile, actualPath, ctx.getType().substring("poll".length()),
- actualWarnCount, plainExecutor);
+ expectedWarnings, plainExecutor);
break;
case "polldynamic":
polldynamic(testCaseCtx, ctx, variableCtx, statement, isDmlRecoveryTest, pb, cUnit, queryCount,
- expectedResultFileCtxs, testFile, actualPath, actualWarnCount);
+ expectedResultFileCtxs, testFile, actualPath, expectedWarnings);
break;
case "query":
case "async":
@@ -1004,11 +1013,7 @@
variableCtx, ctx, expectedResultFile, actualResultFile, queryCount,
expectedResultFileCtxs.size(), cUnit.getParameter(), ComparisonEnum.TEXT);
- if (testCaseCtx.getTestCase().isCheckWarnings()) {
- boolean expectedSourceLoc = testCaseCtx.isSourceLocationExpected(cUnit);
- validateWarnings(extractedResult.getWarnings(), cUnit.getExpectedWarn(), actualWarnCount,
- expectedSourceLoc);
- }
+ validateWarning(extractedResult, testCaseCtx, cUnit, testFile, expectedWarnings);
break;
case "store":
// This is a query that returns the expected output of a subsequent query
@@ -1275,6 +1280,7 @@
final Optional<String> body = extractBody(statement);
final Predicate<Integer> statusCodePredicate = extractStatusCodePredicate(statement);
final boolean extracResult = isExtracResult(statement);
+ final boolean extractStatus = isExtractStatus(statement);
final String mimeReqType = extractHttpRequestType(statement);
ContentType contentType = mimeReqType != null ? ContentType.create(mimeReqType, UTF_8) : TEXT_PLAIN_UTF8;
InputStream resultStream;
@@ -1283,12 +1289,14 @@
} else if ("uri".equals(extension)) {
resultStream = executeURI(reqType, URI.create(variablesReplaced), fmt, params, statusCodePredicate, body,
contentType);
- if (extracResult) {
- resultStream = ResultExtractor.extract(resultStream, UTF_8).getResult();
- }
} else {
throw new IllegalArgumentException("Unexpected format for method " + reqType + ": " + extension);
}
+ if (extracResult) {
+ resultStream = ResultExtractor.extract(resultStream, UTF_8).getResult();
+ } else if (extractStatus) {
+ resultStream = ResultExtractor.extractStatus(resultStream, UTF_8);
+ }
if (handleVar != null) {
String handle = ResultExtractor.extractHandle(resultStream, UTF_8);
if (handle != null) {
@@ -1407,23 +1415,22 @@
FilenameUtils.getExtension(ctx.getFile().getName()));
return executeQuery(fmt, statement, variableCtx, ctx, expectedResultFile, actualResultFile, queryCount,
numResultFiles, params, compare, uri);
-
}
private void polldynamic(TestCaseContext testCaseCtx, TestFileContext ctx, Map<String, Object> variableCtx,
String statement, boolean isDmlRecoveryTest, ProcessBuilder pb, CompilationUnit cUnit,
MutableInt queryCount, List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath,
- MutableInt actualWarnCount) throws Exception {
+ BitSet expectedWarnings) throws Exception {
IExpectedResultPoller poller = getExpectedResultPoller(statement);
final String key = getKey(statement);
poll(testCaseCtx, ctx, variableCtx, statement, isDmlRecoveryTest, pb, cUnit, queryCount, expectedResultFileCtxs,
- testFile, actualPath, "validate", actualWarnCount, new IPollTask() {
+ testFile, actualPath, "validate", expectedWarnings, new IPollTask() {
@Override
public void execute(TestCaseContext testCaseCtx, TestFileContext ctx,
Map<String, Object> variableCtx, String statement, boolean isDmlRecoveryTest,
ProcessBuilder pb, CompilationUnit cUnit, MutableInt queryCount,
List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath,
- MutableInt actualWarnCount) throws Exception {
+ BitSet expectedWarnings) throws Exception {
File actualResultFile = new File(actualPath, testCaseCtx.getTestCase().getFilePath()
+ File.separatorChar + cUnit.getName() + '.' + ctx.getSeqNum() + ".polled.adm");
if (actualResultFile.exists() && !actualResultFile.delete()) {
@@ -1461,7 +1468,7 @@
private void poll(TestCaseContext testCaseCtx, TestFileContext ctx, Map<String, Object> variableCtx,
String statement, boolean isDmlRecoveryTest, ProcessBuilder pb, CompilationUnit cUnit,
MutableInt queryCount, List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath,
- String newType, MutableInt actualWarnCount, IPollTask pollTask) throws Exception {
+ String newType, BitSet expectedWarnings, IPollTask pollTask) throws Exception {
// polltimeoutsecs=nnn, polldelaysecs=nnn
int timeoutSecs = getTimeoutSecs(statement);
int retryDelaySecs = getRetryDelaySecs(statement);
@@ -1484,7 +1491,7 @@
try {
startSemaphore.release();
pollTask.execute(testCaseCtx, ctx, variableCtx, statement, isDmlRecoveryTest, pb, cUnit,
- queryCount, expectedResultFileCtxs, testFile, actualPath, actualWarnCount);
+ queryCount, expectedResultFileCtxs, testFile, actualPath, expectedWarnings);
} finally {
endSemaphore.release();
}
@@ -1552,30 +1559,40 @@
}
}
- public InputStream executeSqlppUpdateOrDdl(String statement, OutputFormat outputFormat) throws Exception {
+ public ExtractedResult executeSqlppUpdateOrDdl(String statement, OutputFormat outputFormat) throws Exception {
return executeUpdateOrDdl(statement, outputFormat, getQueryServiceUri(SQLPP));
}
- private InputStream executeAqlUpdateOrDdl(String statement, OutputFormat outputFormat) throws Exception {
+ private ExtractedResult executeAqlUpdateOrDdl(String statement, OutputFormat outputFormat) throws Exception {
return executeUpdateOrDdl(statement, outputFormat, getQueryServiceUri(AQL));
}
- private InputStream executeUpdateOrDdl(String statement, OutputFormat outputFormat, URI serviceUri)
+ private ExtractedResult executeUpdateOrDdl(String statement, OutputFormat outputFormat, URI serviceUri)
throws Exception {
- InputStream resultStream = executeQueryService(statement, serviceUri, outputFormat, UTF_8);
- return ResultExtractor.extract(resultStream, UTF_8).getResult();
+ try (InputStream resultStream = executeQueryService(statement, serviceUri, outputFormat, UTF_8)) {
+ return ResultExtractor.extract(resultStream, UTF_8);
+ }
}
protected static boolean isExpected(Exception e, CompilationUnit cUnit) {
final List<String> expErrors = cUnit.getExpectedError();
for (String exp : expErrors) {
- if (e.toString().contains(exp)) {
+ if (e.toString().contains(exp) || containsPattern(e.toString(), exp)) {
return true;
}
}
return false;
}
+ private static boolean containsPattern(String exception, String maybePattern) {
+ try {
+ return Pattern.compile(maybePattern).matcher(exception).find();
+ } catch (PatternSyntaxException pse) {
+ // ignore, this isn't always a legal pattern
+ return false;
+ }
+ }
+
public int getTimeoutSecs(String statement) {
final Matcher timeoutMatcher = POLL_TIMEOUT_PATTERN.matcher(statement);
if (timeoutMatcher.find()) {
@@ -1697,6 +1714,11 @@
return m.find() ? Boolean.valueOf(m.group(1)) : false;
}
+ private static boolean isExtractStatus(String statement) {
+ Matcher m = EXTRACT_STATUS_PATTERN.matcher(statement);
+ return m.find();
+ }
+
private static boolean isJsonEncoded(String httpRequestType) throws Exception {
if (httpRequestType == null || httpRequestType.isEmpty()) {
return true;
@@ -1727,8 +1749,7 @@
protected InputStream executeHttp(String ctxType, String endpoint, OutputFormat fmt, List<Parameter> params,
Predicate<Integer> statusCodePredicate, Optional<String> body, ContentType contentType) throws Exception {
- String[] split = endpoint.split("\\?");
- URI uri = createEndpointURI(split[0], split.length > 1 ? split[1] : null);
+ URI uri = createEndpointURI(endpoint);
return executeURI(ctxType, uri, fmt, params, statusCodePredicate, body, contentType);
}
@@ -1745,7 +1766,7 @@
//get node process id
OutputFormat fmt = OutputFormat.CLEAN_JSON;
String endpoint = "/admin/cluster/node/" + nodeId + "/config";
- InputStream executeJSONGet = executeJSONGet(fmt, createEndpointURI(endpoint, null));
+ InputStream executeJSONGet = executeJSONGet(fmt, createEndpointURI(endpoint));
StringWriter actual = new StringWriter();
IOUtils.copy(executeJSONGet, actual, UTF_8);
String config = actual.toString();
@@ -1775,7 +1796,7 @@
private void deleteNCTxnLogs(String nodeId, CompilationUnit cUnit) throws Exception {
OutputFormat fmt = OutputFormat.forCompilationUnit(cUnit);
String endpoint = "/admin/cluster/node/" + nodeId + "/config";
- InputStream executeJSONGet = executeJSONGet(fmt, createEndpointURI(endpoint, null));
+ InputStream executeJSONGet = executeJSONGet(fmt, createEndpointURI(endpoint));
StringWriter actual = new StringWriter();
IOUtils.copy(executeJSONGet, actual, UTF_8);
String config = actual.toString();
@@ -1797,8 +1818,8 @@
List<CompilationUnit> cUnits = testCaseCtx.getTestCase().getCompilationUnit();
for (CompilationUnit cUnit : cUnits) {
List<String> expectedErrors = cUnit.getExpectedError();
- int expectedWarnCount = cUnit.getExpectedWarn().size();
- MutableInt actualWarnCount = new MutableInt(0);
+ BitSet expectedWarnings = new BitSet(cUnit.getExpectedWarn().size());
+ expectedWarnings.set(0, cUnit.getExpectedWarn().size());
LOGGER.info(
"Starting [TEST]: " + testCaseCtx.getTestCase().getFilePath() + "/" + cUnit.getName() + " ... ");
Map<String, Object> variableCtx = new HashMap<>();
@@ -1818,7 +1839,7 @@
try {
if (!testFile.getName().startsWith(DIAGNOSE)) {
executeTestFile(testCaseCtx, ctx, variableCtx, statement, isDmlRecoveryTest, pb, cUnit,
- queryCount, expectedResultFileCtxs, testFile, actualPath, actualWarnCount);
+ queryCount, expectedResultFileCtxs, testFile, actualPath, expectedWarnings);
}
} catch (TestLoop loop) {
// rewind the iterator until we find our target
@@ -1850,7 +1871,7 @@
throw new Exception(
"Test \"" + cUnit.getName() + "\" FAILED; expected exception was not thrown...");
}
- ensureWarnings(actualWarnCount.getValue(), expectedWarnCount, cUnit);
+ ensureWarnings(expectedWarnings, cUnit);
LOGGER.info(
"[TEST]: " + testCaseCtx.getTestCase().getFilePath() + "/" + cUnit.getName() + " PASSED ");
if (passedGroup != null) {
@@ -1873,7 +1894,7 @@
final File file = ctx.getFile();
final String statement = readTestFile(file);
executeTestFile(testCaseCtx, ctx, variableCtx, statement, false, pb, cUnit, new MutableInt(-1),
- Collections.emptyList(), file, null, new MutableInt(-1));
+ Collections.emptyList(), file, null, new BitSet());
}
}
} catch (Exception diagnosticFailure) {
@@ -1892,7 +1913,7 @@
// Get the expected exception
expectedError = expectedErrors.get(numOfErrors - 1);
String actualError = e.toString();
- if (!actualError.contains(expectedError)) {
+ if (!actualError.contains(expectedError) && !containsPattern(actualError, expectedError)) {
LOGGER.error("Expected to find the following in error text: +++++{}+++++", expectedError);
return true;
}
@@ -1925,17 +1946,17 @@
+ cUnit.getName() + "_qbc.adm");
}
- protected URI createEndpointURI(String path, String query) throws URISyntaxException {
+ protected URI createEndpointURI(String pathAndQuery) throws URISyntaxException {
InetSocketAddress endpoint;
- if (!ncEndPointsList.isEmpty() && path.equals(Servlets.QUERY_SERVICE)) {
+ if (!ncEndPointsList.isEmpty() && pathAndQuery.equals(Servlets.QUERY_SERVICE)) {
int endpointIdx = Math.abs(endpointSelector++ % ncEndPointsList.size());
endpoint = ncEndPointsList.get(endpointIdx);
- } else if (isCcEndPointPath(path)) {
+ } else if (isCcEndPointPath(pathAndQuery)) {
int endpointIdx = Math.abs(endpointSelector++ % endpoints.size());
endpoint = endpoints.get(endpointIdx);
} else {
// allowed patterns: [nc:endpointName URL] or [nc:nodeId:port URL]
- final String[] tokens = path.split(" ");
+ final String[] tokens = pathAndQuery.split(" ");
if (tokens.length != 2) {
throw new IllegalArgumentException("Unrecognized http pattern");
}
@@ -1949,19 +1970,19 @@
} else {
endpoint = getNcEndPoint(endpointName);
}
- path = tokens[1];
+ pathAndQuery = tokens[1];
}
- URI uri = new URI("http", null, endpoint.getHostString(), endpoint.getPort(), path, query, null);
+ URI uri = URI.create("http://" + toHostPort(endpoint.getHostString(), endpoint.getPort()) + pathAndQuery);
LOGGER.debug("Created endpoint URI: " + uri);
return uri;
}
public URI getEndpoint(String servlet) throws URISyntaxException {
- return createEndpointURI(Servlets.getAbsolutePath(getPath(servlet)), null);
+ return createEndpointURI(Servlets.getAbsolutePath(getPath(servlet)));
}
public URI getEndpoint(String servlet, String extension) throws URISyntaxException {
- return createEndpointURI(Servlets.getAbsolutePath(getPath(servlet)), null);
+ return createEndpointURI(Servlets.getAbsolutePath(getPath(servlet)));
}
public static String stripJavaComments(String text) {
@@ -2074,9 +2095,17 @@
LOGGER.info("Cluster state now " + desiredState);
}
- protected void ensureWarnings(int actualWarnCount, int expectedWarnCount, CompilationUnit cUnit) throws Exception {
- if (actualWarnCount < expectedWarnCount) {
- LOGGER.error("Test {} failed to raise (an) expected warning(s)", cUnit.getName());
+ protected void ensureWarnings(BitSet expectedWarnings, CompilationUnit cUnit) throws Exception {
+ boolean fail = !expectedWarnings.isEmpty();
+ if (fail) {
+ LOGGER.error("Test {} failed to raise (an) expected warning(s):", cUnit.getName());
+ }
+ List<String> expectedWarn = cUnit.getExpectedWarn();
+ for (int i = expectedWarnings.nextSetBit(0); i >= 0; i = expectedWarnings.nextSetBit(i + 1)) {
+ String warning = expectedWarn.get(i);
+ LOGGER.error(warning);
+ }
+ if (fail) {
throw new Exception("Test \"" + cUnit.getName() + "\" FAILED; expected warning(s) was not returned...");
}
}
@@ -2212,22 +2241,40 @@
return extension.endsWith(AQL) ? getEndpoint(Servlets.QUERY_AQL) : getEndpoint(Servlets.QUERY_SERVICE);
}
- private void validateWarnings(List<String> actualWarnings, List<String> expectedWarn, MutableInt actualWarnCount,
- boolean expectedSourceLoc) throws Exception {
+ protected void validateWarning(ExtractedResult result, TestCaseContext testCaseCtx, CompilationUnit cUnit,
+ File testFile, BitSet expectedWarnings) throws Exception {
+ if (testCaseCtx.getTestCase().isCheckWarnings()) {
+ boolean expectedSourceLoc = testCaseCtx.isSourceLocationExpected(cUnit);
+ validateWarnings(result.getWarnings(), cUnit.getExpectedWarn(), expectedWarnings, expectedSourceLoc,
+ testFile);
+ }
+ }
+
+ protected void validateWarnings(List<String> actualWarnings, List<String> expectedWarn, BitSet expectedWarnings,
+ boolean expectedSourceLoc, File testFile) throws Exception {
if (actualWarnings != null) {
for (String actualWarn : actualWarnings) {
- if (expectedWarn.stream().noneMatch(actualWarn::contains)) {
- throw new Exception("unexpected warning was encountered (" + actualWarn + ")");
+ OptionalInt first = IntStream.range(0, expectedWarn.size())
+ .filter(i -> actualWarn.contains(expectedWarn.get(i)) && expectedWarnings.get(i)).findFirst();
+ if (!first.isPresent()) {
+ String msg = "unexpected warning was encountered or has already been matched (" + actualWarn + ")";
+ LOGGER.error(msg);
+ if (!expectedWarnings.isEmpty()) {
+ LOGGER.error("was expecting the following warnings: ");
+ }
+ for (int i = expectedWarnings.nextSetBit(0); i >= 0; i = expectedWarnings.nextSetBit(i + 1)) {
+ LOGGER.error(expectedWarn.get(i));
+ }
+ throw new Exception(msg);
}
if (expectedSourceLoc && !containsSourceLocation(actualWarn)) {
throw new Exception(MessageFormat.format(
"Expected to find source location \"{}, {}\" in warning text: +++++{}+++++",
ERR_MSG_SRC_LOC_LINE_REGEX, ERR_MSG_SRC_LOC_COLUMN_REGEX, actualWarn));
}
- actualWarnCount.increment();
- if (actualWarnCount.getValue() > expectedWarn.size()) {
- throw new Exception("returned warnings exceeded expected warnings");
- }
+ int warningIndex = first.getAsInt();
+ expectedWarnings.clear(warningIndex);
+ LOGGER.info("testFile {} issued an (expected) warning", testFile);
}
}
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java
index 03ca1f0..dd0bd9a 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java
@@ -23,7 +23,9 @@
import java.rmi.RemoteException;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.HashSet;
import java.util.List;
+import java.util.Set;
import java.util.concurrent.Semaphore;
import org.apache.asterix.app.bootstrap.TestNodeController;
@@ -62,6 +64,7 @@
import org.apache.hyracks.storage.am.lsm.btree.impl.ITestOpCallback;
import org.apache.hyracks.storage.am.lsm.btree.impl.TestLsmBtree;
import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMComponentId;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMDiskComponent;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
@@ -146,8 +149,7 @@
checkComponentIds();
// insert more records
createInsertOps();
- insertRecords(PARTITION_0, StorageTestUtils.RECORDS_PER_COMPONENT, StorageTestUtils.RECORDS_PER_COMPONENT,
- true);
+ insertRecords(PARTITION_0, StorageTestUtils.TOTAL_NUM_OF_RECORDS, StorageTestUtils.RECORDS_PER_COMPONENT, true);
dsInfo.waitForIO();
checkComponentIds();
@@ -486,8 +488,14 @@
List<ILSMDiskComponent> secondaryDiskComponents = secondaryIndexes[partitionIndex].getDiskComponents();
Assert.assertEquals(primaryDiskComponents.size(), secondaryDiskComponents.size());
+ Set<ILSMComponentId> uniqueIds = new HashSet<>();
for (int i = 0; i < primaryDiskComponents.size(); i++) {
Assert.assertEquals(primaryDiskComponents.get(i).getId(), secondaryDiskComponents.get(i).getId());
+ ILSMComponentId id = primaryDiskComponents.get(i).getId();
+ boolean added = uniqueIds.add(id);
+ if (!added) {
+ throw new IllegalStateException("found duplicate component ids: " + id);
+ }
}
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/AwsS3ExternalDatasetOnePartitionTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/AwsS3ExternalDatasetOnePartitionTest.java
new file mode 100644
index 0000000..48e966e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/AwsS3ExternalDatasetOnePartitionTest.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.external_dataset.aws;
+
+import java.util.Collection;
+
+import org.apache.asterix.test.runtime.LangExecutionUtil;
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.junit.FixMethodOrder;
+import org.junit.runner.RunWith;
+import org.junit.runners.MethodSorters;
+import org.junit.runners.Parameterized;
+
+/**
+ * Runs an AWS S3 mock server and test it as an external dataset using one node one partition.
+ */
+@RunWith(Parameterized.class)
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+public class AwsS3ExternalDatasetOnePartitionTest extends AwsS3ExternalDatasetTest {
+
+ public AwsS3ExternalDatasetOnePartitionTest(TestCaseContext tcCtx) {
+ super(tcCtx);
+ }
+
+ @Parameterized.Parameters(name = "AwsS3ExternalDatasetOnePartitionTest {index}: {0}")
+ public static Collection<Object[]> tests() throws Exception {
+ SUITE_TESTS = "testsuite_external_dataset_one_partition.xml";
+ ONLY_TESTS = "only_external_dataset.xml";
+ TEST_CONFIG_FILE_NAME = "src/test/resources/cc-single.conf";
+ PREPARE_BUCKET = AwsS3ExternalDatasetOnePartitionTest::prepareS3Bucket;
+ PREPARE_FIXED_DATA_BUCKET = AwsS3ExternalDatasetOnePartitionTest::prepareFixedDataBucket;
+ PREPARE_MIXED_DATA_BUCKET = AwsS3ExternalDatasetOnePartitionTest::prepareMixedDataBucket;
+ return LangExecutionUtil.tests(ONLY_TESTS, SUITE_TESTS);
+ }
+
+ private static void prepareS3Bucket() {
+ }
+
+ private static void prepareFixedDataBucket() {
+ }
+
+ private static void prepareMixedDataBucket() {
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/AwsS3ExternalDatasetTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/AwsS3ExternalDatasetTest.java
new file mode 100644
index 0000000..502a7eb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/AwsS3ExternalDatasetTest.java
@@ -0,0 +1,591 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.external_dataset.aws;
+
+import static org.apache.hyracks.util.file.FileUtil.joinPath;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.zip.GZIPOutputStream;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.test.runtime.ExecutionTestUtil;
+import org.apache.asterix.test.runtime.LangExecutionUtil;
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.asterix.testframework.context.TestFileContext;
+import org.apache.asterix.testframework.xml.TestCase;
+import org.apache.commons.io.FilenameUtils;
+import org.apache.commons.lang3.mutable.MutableInt;
+import org.apache.hyracks.control.nc.NodeControllerService;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.FixMethodOrder;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.MethodSorters;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import io.findify.s3mock.S3Mock;
+import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider;
+import software.amazon.awssdk.core.sync.RequestBody;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.s3.S3Client;
+import software.amazon.awssdk.services.s3.S3ClientBuilder;
+import software.amazon.awssdk.services.s3.model.CreateBucketRequest;
+import software.amazon.awssdk.services.s3.model.DeleteBucketRequest;
+import software.amazon.awssdk.services.s3.model.NoSuchBucketException;
+import software.amazon.awssdk.services.s3.model.PutObjectRequest;
+
+/**
+ * Runs an AWS S3 mock server and test it as an external dataset
+ */
+@RunWith(Parameterized.class)
+@FixMethodOrder(MethodSorters.NAME_ASCENDING)
+public class AwsS3ExternalDatasetTest {
+
+ private static final Logger LOGGER = LogManager.getLogger();
+
+ // subclasses of this class MUST instantiate these variables before using them to avoid unexpected behavior
+ static String SUITE_TESTS;
+ static String ONLY_TESTS;
+ static String TEST_CONFIG_FILE_NAME;
+ static Runnable PREPARE_BUCKET;
+ static Runnable PREPARE_FIXED_DATA_BUCKET;
+ static Runnable PREPARE_MIXED_DATA_BUCKET;
+
+ // Base directory paths for data files
+ private static final String JSON_DATA_PATH = joinPath("data", "json");
+ private static final String CSV_DATA_PATH = joinPath("data", "csv");
+ private static final String TSV_DATA_PATH = joinPath("data", "tsv");
+ private static final String MIXED_DATA_PATH = joinPath("data", "mixed");
+
+ // Service endpoint
+ private static final int MOCK_SERVER_PORT = 8001;
+ private static final String MOCK_SERVER_HOSTNAME = "http://localhost:" + MOCK_SERVER_PORT;
+
+ // Region, bucket and definitions
+ private static final String MOCK_SERVER_REGION = "us-west-2";
+ private static final String MOCK_SERVER_BUCKET = "playground";
+ private static final String FIXED_DATA_BUCKET = "fixed-data"; // Do not use, has fixed data
+ private static final String INCLUDE_EXCLUDE_BUCKET = "include-exclude"; // include & exclude bucket
+ private static final String JSON_DEFINITION = "json-data/reviews/"; // data resides here
+ private static final String CSV_DEFINITION = "csv-data/reviews/"; // data resides here
+ private static final String TSV_DEFINITION = "tsv-data/reviews/"; // data resides here
+
+ // This is used for a test to generate over 1000 number of files
+ private static final String OVER_1000_OBJECTS_PATH = "over-1000-objects";
+ private static final int OVER_1000_OBJECTS_COUNT = 2999;
+
+ private static final Set<String> fileNames = new HashSet<>();
+ private static final CreateBucketRequest.Builder CREATE_BUCKET_BUILDER = CreateBucketRequest.builder();
+ private static final DeleteBucketRequest.Builder DELETE_BUCKET_BUILDER = DeleteBucketRequest.builder();
+ private static final PutObjectRequest.Builder PUT_OBJECT_BUILDER = PutObjectRequest.builder();
+
+ // IMPORTANT: The following values must be used in the AWS S3 test case
+ private static S3Mock s3MockServer;
+ private static S3Client client;
+ private static final PutObjectRequest.Builder builder = PutObjectRequest.builder().bucket(MOCK_SERVER_BUCKET);
+ private static final PutObjectRequest.Builder includeExcludeBuilder =
+ PutObjectRequest.builder().bucket(INCLUDE_EXCLUDE_BUCKET);
+
+ protected TestCaseContext tcCtx;
+
+ public AwsS3ExternalDatasetTest(TestCaseContext tcCtx) {
+ this.tcCtx = tcCtx;
+ }
+
+ @BeforeClass
+ public static void setUp() throws Exception {
+ final TestExecutor testExecutor = new AwsTestExecutor();
+ LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME, testExecutor);
+ setNcEndpoints(testExecutor);
+ startAwsS3MockServer();
+ }
+
+ @AfterClass
+ public static void tearDown() throws Exception {
+ LangExecutionUtil.tearDown();
+
+ // Shutting down S3 mock server
+ LOGGER.info("Shutting down S3 mock server and client");
+ if (client != null) {
+ client.close();
+ }
+ if (s3MockServer != null) {
+ s3MockServer.shutdown();
+ }
+ LOGGER.info("S3 mock down and client shut down successfully");
+ }
+
+ @Parameters(name = "AwsS3ExternalDatasetTest {index}: {0}")
+ public static Collection<Object[]> tests() throws Exception {
+ SUITE_TESTS = "testsuite_external_dataset.xml";
+ ONLY_TESTS = "only_external_dataset.xml";
+ TEST_CONFIG_FILE_NAME = "src/main/resources/cc.conf";
+ PREPARE_BUCKET = AwsS3ExternalDatasetTest::prepareS3Bucket;
+ PREPARE_FIXED_DATA_BUCKET = AwsS3ExternalDatasetTest::prepareFixedDataBucket;
+ PREPARE_MIXED_DATA_BUCKET = AwsS3ExternalDatasetTest::prepareMixedDataBucket;
+ return LangExecutionUtil.tests(ONLY_TESTS, SUITE_TESTS);
+ }
+
+ @Test
+ public void test() throws Exception {
+ LangExecutionUtil.test(tcCtx);
+ }
+
+ private static void setNcEndpoints(TestExecutor testExecutor) {
+ final NodeControllerService[] ncs = ExecutionTestUtil.integrationUtil.ncs;
+ final Map<String, InetSocketAddress> ncEndPoints = new HashMap<>();
+ final String ip = InetAddress.getLoopbackAddress().getHostAddress();
+ for (NodeControllerService nc : ncs) {
+ final String nodeId = nc.getId();
+ final INcApplicationContext appCtx = (INcApplicationContext) nc.getApplicationContext();
+ int apiPort = appCtx.getExternalProperties().getNcApiPort();
+ ncEndPoints.put(nodeId, InetSocketAddress.createUnresolved(ip, apiPort));
+ }
+ testExecutor.setNcEndPoints(ncEndPoints);
+ }
+
+ /**
+ * Starts the AWS s3 mocking server and loads some files for testing
+ */
+ private static void startAwsS3MockServer() {
+ // Starting S3 mock server to be used instead of real S3 server
+ LOGGER.info("Starting S3 mock server");
+ s3MockServer = new S3Mock.Builder().withPort(MOCK_SERVER_PORT).withInMemoryBackend().build();
+ s3MockServer.start();
+ LOGGER.info("S3 mock server started successfully");
+
+ // Create a client and add some files to the S3 mock server
+ LOGGER.info("Creating S3 client to load initial files to S3 mock server");
+ S3ClientBuilder builder = S3Client.builder();
+ URI endpoint = URI.create(MOCK_SERVER_HOSTNAME); // endpoint pointing to S3 mock server
+ builder.region(Region.of(MOCK_SERVER_REGION)).credentialsProvider(AnonymousCredentialsProvider.create())
+ .endpointOverride(endpoint);
+ client = builder.build();
+ LOGGER.info("Client created successfully");
+
+ // Create the bucket and upload some json files
+ PREPARE_BUCKET.run();
+ PREPARE_FIXED_DATA_BUCKET.run();
+ PREPARE_MIXED_DATA_BUCKET.run();
+ }
+
+ /**
+ * Creates a bucket and fills it with some files for testing purpose.
+ */
+ private static void prepareS3Bucket() {
+ LOGGER.info("creating bucket " + MOCK_SERVER_BUCKET);
+ client.createBucket(CreateBucketRequest.builder().bucket(MOCK_SERVER_BUCKET).build());
+ LOGGER.info("bucket created successfully");
+
+ LOGGER.info("Adding JSON files to the bucket");
+ loadJsonFiles();
+ LOGGER.info("JSON Files added successfully");
+
+ LOGGER.info("Adding CSV files to the bucket");
+ loadCsvFiles();
+ LOGGER.info("CSV Files added successfully");
+
+ LOGGER.info("Adding TSV files to the bucket");
+ loadTsvFiles();
+ LOGGER.info("TSV Files added successfully");
+
+ LOGGER.info("Loading " + OVER_1000_OBJECTS_COUNT + " into " + OVER_1000_OBJECTS_PATH);
+ loadLargeNumberOfFiles();
+ LOGGER.info("Added " + OVER_1000_OBJECTS_COUNT + " files into " + OVER_1000_OBJECTS_PATH + " successfully");
+ }
+
+ /**
+ * This bucket is being filled by fixed data, a test is counting all records in this bucket. If this bucket is
+ * changed, the test case will fail and its result will need to be updated each time
+ */
+ private static void prepareFixedDataBucket() {
+ LOGGER.info("creating bucket " + FIXED_DATA_BUCKET);
+ client.createBucket(CreateBucketRequest.builder().bucket(FIXED_DATA_BUCKET).build());
+ LOGGER.info("bucket " + FIXED_DATA_BUCKET + " created successfully");
+
+ LOGGER.info("Loading fixed data to " + FIXED_DATA_BUCKET);
+
+ // Files data
+ RequestBody requestBody = RequestBody.fromFile(Paths.get(JSON_DATA_PATH, "single-line", "20-records.json"));
+ client.putObject(builder.bucket(FIXED_DATA_BUCKET).key("1.json").build(), requestBody);
+ client.putObject(builder.bucket(FIXED_DATA_BUCKET).key("2.json").build(), requestBody);
+ client.putObject(builder.bucket(FIXED_DATA_BUCKET).key("lvl1/3.json").build(), requestBody);
+ client.putObject(builder.bucket(FIXED_DATA_BUCKET).key("lvl1/4.json").build(), requestBody);
+ client.putObject(builder.bucket(FIXED_DATA_BUCKET).key("lvl1/lvl2/5.json").build(), requestBody);
+ }
+
+ private static void loadJsonFiles() {
+ String dataBasePath = JSON_DATA_PATH;
+ String definition = JSON_DEFINITION;
+
+ // Normal format
+ String definitionSegment = "json";
+ loadData(dataBasePath, "single-line", "20-records.json", definition, definitionSegment, false);
+ loadData(dataBasePath, "multi-lines", "20-records.json", definition, definitionSegment, false);
+ loadData(dataBasePath, "multi-lines-with-arrays", "5-records.json", definition, definitionSegment, false);
+ loadData(dataBasePath, "multi-lines-with-nested-objects", "5-records.json", definition, definitionSegment,
+ false);
+
+ definitionSegment = "json-array-of-objects";
+ loadData(dataBasePath, "single-line", "array_of_objects.json", "json-data/", definitionSegment, false, false);
+
+ // gz compressed format
+ definitionSegment = "gz";
+ loadGzData(dataBasePath, "single-line", "20-records.json", definition, definitionSegment, false);
+ loadGzData(dataBasePath, "multi-lines", "20-records.json", definition, definitionSegment, false);
+ loadGzData(dataBasePath, "multi-lines-with-arrays", "5-records.json", definition, definitionSegment, false);
+ loadGzData(dataBasePath, "multi-lines-with-nested-objects", "5-records.json", definition, definitionSegment,
+ false);
+
+ // Mixed normal and gz compressed format
+ definitionSegment = "mixed";
+ loadData(dataBasePath, "single-line", "20-records.json", definition, definitionSegment, false);
+ loadData(dataBasePath, "multi-lines", "20-records.json", definition, definitionSegment, false);
+ loadData(dataBasePath, "multi-lines-with-arrays", "5-records.json", definition, definitionSegment, false);
+ loadData(dataBasePath, "multi-lines-with-nested-objects", "5-records.json", definition, definitionSegment,
+ false);
+ loadGzData(dataBasePath, "single-line", "20-records.json", definition, definitionSegment, false);
+ loadGzData(dataBasePath, "multi-lines", "20-records.json", definition, definitionSegment, false);
+ loadGzData(dataBasePath, "multi-lines-with-arrays", "5-records.json", definition, definitionSegment, false);
+ loadGzData(dataBasePath, "multi-lines-with-nested-objects", "5-records.json", definition, definitionSegment,
+ false);
+ }
+
+ private static void loadCsvFiles() {
+ String dataBasePath = CSV_DATA_PATH;
+ String definition = CSV_DEFINITION;
+
+ // Normal format
+ String definitionSegment = "csv";
+ loadData(dataBasePath, "", "01.csv", definition, definitionSegment, false);
+ loadData(dataBasePath, "", "02.csv", definition, definitionSegment, false);
+
+ // gz compressed format
+ definitionSegment = "gz";
+ loadGzData(dataBasePath, "", "01.csv", definition, definitionSegment, false);
+ loadGzData(dataBasePath, "", "02.csv", definition, definitionSegment, false);
+
+ // Mixed normal and gz compressed format
+ definitionSegment = "mixed";
+ loadData(dataBasePath, "", "01.csv", definition, definitionSegment, false);
+ loadData(dataBasePath, "", "02.csv", definition, definitionSegment, false);
+ loadGzData(dataBasePath, "", "01.csv", definition, definitionSegment, false);
+ loadGzData(dataBasePath, "", "02.csv", definition, definitionSegment, false);
+ }
+
+ private static void loadTsvFiles() {
+ String dataBasePath = TSV_DATA_PATH;
+ String definition = TSV_DEFINITION;
+
+ // Normal format
+ String definitionSegment = "tsv";
+ loadData(dataBasePath, "", "01.tsv", definition, definitionSegment, false);
+ loadData(dataBasePath, "", "02.tsv", definition, definitionSegment, false);
+
+ // gz compressed format
+ definitionSegment = "gz";
+ loadGzData(dataBasePath, "", "01.tsv", definition, definitionSegment, false);
+ loadGzData(dataBasePath, "", "02.tsv", definition, definitionSegment, false);
+
+ // Mixed normal and gz compressed format
+ definitionSegment = "mixed";
+ loadData(dataBasePath, "", "01.tsv", definition, definitionSegment, false);
+ loadData(dataBasePath, "", "02.tsv", definition, definitionSegment, false);
+ loadGzData(dataBasePath, "", "01.tsv", definition, definitionSegment, false);
+ loadGzData(dataBasePath, "", "02.tsv", definition, definitionSegment, false);
+ }
+
+ private static void loadData(String fileBasePath, String filePathSegment, String filename, String definition,
+ String definitionSegment, boolean removeExtension) {
+ loadData(fileBasePath, filePathSegment, filename, definition, definitionSegment, removeExtension, true);
+ }
+
+ private static void loadData(String fileBasePath, String filePathSegment, String filename, String definition,
+ String definitionSegment, boolean removeExtension, boolean copyToSubLevels) {
+ // Files data
+ Path filePath = Paths.get(fileBasePath, filePathSegment, filename);
+ RequestBody requestBody = RequestBody.fromFile(filePath);
+
+ // Keep or remove the file extension
+ Assert.assertFalse("Files with no extension are not supported yet for external datasets", removeExtension);
+ String finalFileName;
+ if (removeExtension) {
+ finalFileName = FilenameUtils.removeExtension(filename);
+ } else {
+ finalFileName = filename;
+ }
+
+ // Files base definition
+ filePathSegment = filePathSegment.isEmpty() ? "" : filePathSegment + "/";
+ definitionSegment = definitionSegment.isEmpty() ? "" : definitionSegment + "/";
+ String basePath = definition + filePathSegment + definitionSegment;
+
+ // Load the data
+ client.putObject(builder.key(basePath + finalFileName).build(), requestBody);
+ if (copyToSubLevels) {
+ client.putObject(builder.key(basePath + "level1a/" + finalFileName).build(), requestBody);
+ client.putObject(builder.key(basePath + "level1b/" + finalFileName).build(), requestBody);
+ client.putObject(builder.key(basePath + "level1a/level2a/" + finalFileName).build(), requestBody);
+ client.putObject(builder.key(basePath + "level1a/level2b/" + finalFileName).build(), requestBody);
+ }
+ }
+
+ private static void loadGzData(String fileBasePath, String filePathSegment, String filename, String definition,
+ String definitionSegment, boolean removeExtension) {
+ try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
+ GZIPOutputStream gzipOutputStream = new GZIPOutputStream(byteArrayOutputStream)) {
+
+ // Files data
+ Path filePath = Paths.get(fileBasePath, filePathSegment, filename);
+
+ // Get the compressed data
+ gzipOutputStream.write(Files.readAllBytes(filePath));
+ gzipOutputStream.close(); // Need to close or data will be invalid
+ byte[] gzipBytes = byteArrayOutputStream.toByteArray();
+ RequestBody requestBody = RequestBody.fromBytes(gzipBytes);
+
+ // Keep or remove the file extension
+ Assert.assertFalse("Files with no extension are not supported yet for external datasets", removeExtension);
+ String finalFileName;
+ if (removeExtension) {
+ finalFileName = FilenameUtils.removeExtension(filename);
+ } else {
+ finalFileName = filename;
+ }
+ finalFileName += ".gz";
+
+ // Files base definition
+ filePathSegment = filePathSegment.isEmpty() ? "" : filePathSegment + "/";
+ definitionSegment = definitionSegment.isEmpty() ? "" : definitionSegment + "/";
+ String basePath = definition + filePathSegment + definitionSegment;
+
+ // Load the data
+ client.putObject(builder.key(basePath + finalFileName).build(), requestBody);
+ client.putObject(builder.key(basePath + "level1a/" + finalFileName).build(), requestBody);
+ client.putObject(builder.key(basePath + "level1b/" + finalFileName).build(), requestBody);
+ client.putObject(builder.key(basePath + "level1a/level2a/" + finalFileName).build(), requestBody);
+ client.putObject(builder.key(basePath + "level1a/level2b/" + finalFileName).build(), requestBody);
+ } catch (Exception ex) {
+ LOGGER.error(ex.getMessage());
+ }
+ }
+
+ /**
+ * Generates over 1000 objects and upload them to S3 mock server, 1 record per object
+ */
+ private static void loadLargeNumberOfFiles() {
+ for (int i = 0; i < OVER_1000_OBJECTS_COUNT; i++) {
+ RequestBody body = RequestBody.fromString("{\"id\":" + i + "}");
+ client.putObject(builder.key(OVER_1000_OBJECTS_PATH + "/" + i + ".json").build(), body);
+ }
+ }
+
+ /**
+ * Loads a combination of different file formats in the same path
+ */
+ private static void prepareMixedDataBucket() {
+ LOGGER.info("creating bucket " + INCLUDE_EXCLUDE_BUCKET);
+ client.createBucket(CreateBucketRequest.builder().bucket(INCLUDE_EXCLUDE_BUCKET).build());
+ LOGGER.info("bucket " + INCLUDE_EXCLUDE_BUCKET + " created successfully");
+
+ // JSON
+ client.putObject(
+ includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/extension/" + "hello-world-2018.json").build(),
+ RequestBody.fromString("{\"id\":" + 1 + "}"));
+ client.putObject(
+ includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/extension/" + "hello-world-2019.json").build(),
+ RequestBody.fromString("{\"id\":" + 2 + "}"));
+ client.putObject(
+ includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/extension/" + "hello-world-2020.json").build(),
+ RequestBody.fromString("{\"id\":" + 3 + "}"));
+ client.putObject(
+ includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/EXTENSION/" + "goodbye-world-2018.json").build(),
+ RequestBody.fromString("{\"id\":" + 4 + "}"));
+ client.putObject(
+ includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/EXTENSION/" + "goodbye-world-2019.json").build(),
+ RequestBody.fromString("{\"id\":" + 5 + "}"));
+ client.putObject(
+ includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/EXTENSION/" + "goodbye-world-2020.json").build(),
+ RequestBody.fromString("{\"id\":" + 6 + "}"));
+
+ // CSV
+ client.putObject(
+ includeExcludeBuilder.key(MIXED_DATA_PATH + "/csv/extension/" + "hello-world-2018.csv").build(),
+ RequestBody.fromString("7,\"good\""));
+ client.putObject(
+ includeExcludeBuilder.key(MIXED_DATA_PATH + "/csv/extension/" + "hello-world-2019.csv").build(),
+ RequestBody.fromString("8,\"good\""));
+ client.putObject(
+ includeExcludeBuilder.key(MIXED_DATA_PATH + "/csv/extension/" + "hello-world-2020.csv").build(),
+ RequestBody.fromString("{9,\"good\""));
+ client.putObject(
+ includeExcludeBuilder.key(MIXED_DATA_PATH + "/csv/EXTENSION/" + "goodbye-world-2018.csv").build(),
+ RequestBody.fromString("10,\"good\""));
+ client.putObject(
+ includeExcludeBuilder.key(MIXED_DATA_PATH + "/csv/EXTENSION/" + "goodbye-world-2019.csv").build(),
+ RequestBody.fromString("11,\"good\""));
+ client.putObject(
+ includeExcludeBuilder.key(MIXED_DATA_PATH + "/csv/EXTENSION/" + "goodbye-world-2020.csv").build(),
+ RequestBody.fromString("12,\"good\""));
+
+ // TSV
+ client.putObject(
+ includeExcludeBuilder.key(MIXED_DATA_PATH + "/tsv/extension/" + "hello-world-2018.tsv").build(),
+ RequestBody.fromString("13\t\"good\""));
+ client.putObject(
+ includeExcludeBuilder.key(MIXED_DATA_PATH + "/tsv/extension/" + "hello-world-2019.tsv").build(),
+ RequestBody.fromString("14\t\"good\""));
+ client.putObject(
+ includeExcludeBuilder.key(MIXED_DATA_PATH + "/tsv/extension/" + "hello-world-2020.tsv").build(),
+ RequestBody.fromString("15\t\"good\""));
+ client.putObject(
+ includeExcludeBuilder.key(MIXED_DATA_PATH + "/tsv/EXTENSION/" + "goodbye-world-2018.tsv").build(),
+ RequestBody.fromString("16\t\"good\""));
+ client.putObject(
+ includeExcludeBuilder.key(MIXED_DATA_PATH + "/tsv/EXTENSION/" + "goodbye-world-2019.tsv").build(),
+ RequestBody.fromString("17\t\"good\""));
+ client.putObject(
+ includeExcludeBuilder.key(MIXED_DATA_PATH + "/tsv/EXTENSION/" + "goodbye-world-2020.tsv").build(),
+ RequestBody.fromString("18\t\"good\""));
+
+ // JSON no extension
+ client.putObject(
+ includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/no-extension/" + "hello-world-2018").build(),
+ RequestBody.fromString("{\"id\":" + 1 + "}"));
+ client.putObject(
+ includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/no-extension/" + "hello-world-2019").build(),
+ RequestBody.fromString("{\"id\":" + 2 + "}"));
+ client.putObject(
+ includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/no-extension/" + "hello-world-2020").build(),
+ RequestBody.fromString("{\"id\":" + 3 + "}"));
+ client.putObject(
+ includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/NO-EXTENSION/" + "goodbye-world-2018").build(),
+ RequestBody.fromString("{\"id\":" + 4 + "}"));
+ client.putObject(
+ includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/NO-EXTENSION/" + "goodbye-world-2019").build(),
+ RequestBody.fromString("{\"id\":" + 5 + "}"));
+ client.putObject(
+ includeExcludeBuilder.key(MIXED_DATA_PATH + "/json/NO-EXTENSION/" + "goodbye-world-2020").build(),
+ RequestBody.fromString("{\"id\":" + 6 + "}"));
+ }
+
+ static class AwsTestExecutor extends TestExecutor {
+
+ public void executeTestFile(TestCaseContext testCaseCtx, TestFileContext ctx, Map<String, Object> variableCtx,
+ String statement, boolean isDmlRecoveryTest, ProcessBuilder pb, TestCase.CompilationUnit cUnit,
+ MutableInt queryCount, List<TestFileContext> expectedResultFileCtxs, File testFile, String actualPath,
+ BitSet expectedWarnings) throws Exception {
+ String[] lines;
+ switch (ctx.getType()) {
+ case "s3bucket":
+ // <bucket> <def> <sub-path:new_fname:src_file1,sub-path:new_fname:src_file2,sub-path:src_file3>
+ lines = TestExecutor.stripAllComments(statement).trim().split("\n");
+ String lastLine = lines[lines.length - 1];
+ String[] command = lastLine.trim().split(" ");
+ int length = command.length;
+ if (length != 3) {
+ throw new Exception("invalid create bucket format");
+ }
+ dropRecreateBucket(command[0], command[1], command[2]);
+ break;
+ default:
+ super.executeTestFile(testCaseCtx, ctx, variableCtx, statement, isDmlRecoveryTest, pb, cUnit,
+ queryCount, expectedResultFileCtxs, testFile, actualPath, expectedWarnings);
+ }
+ }
+ }
+
+ private static void dropRecreateBucket(String bucketName, String definition, String files) {
+ String definitionPath = definition + (definition.endsWith("/") ? "" : "/");
+ String[] fileSplits = files.split(",");
+
+ LOGGER.info("Dropping bucket " + bucketName);
+ try {
+ client.deleteBucket(DELETE_BUCKET_BUILDER.bucket(bucketName).build());
+ } catch (NoSuchBucketException e) {
+ // ignore
+ }
+ LOGGER.info("Creating bucket " + bucketName);
+ client.createBucket(CREATE_BUCKET_BUILDER.bucket(bucketName).build());
+ LOGGER.info("Uploading to bucket " + bucketName + " definition " + definitionPath);
+ fileNames.clear();
+ for (int i = 0; i < fileSplits.length; i++) {
+ String[] s3pathAndSourceFile = fileSplits[i].split(":");
+ int size = s3pathAndSourceFile.length;
+ String path;
+ String sourceFilePath;
+ String uploadedFileName;
+ if (size == 1) {
+ // case: playground json-data/reviews SOURCE_FILE1,SOURCE_FILE2
+ path = definitionPath;
+ sourceFilePath = s3pathAndSourceFile[0];
+ uploadedFileName = FilenameUtils.getName(s3pathAndSourceFile[0]);
+ } else if (size == 2) {
+ // case: playground json-data/reviews level1/sub-level:SOURCE_FILE1,level2/sub-level:SOURCE_FILE2
+ String subPathOrNewFileName = s3pathAndSourceFile[0];
+ if (subPathOrNewFileName.startsWith("$$")) {
+ path = definitionPath;
+ sourceFilePath = s3pathAndSourceFile[1];
+ uploadedFileName = subPathOrNewFileName.substring(2);
+ } else {
+ path = definitionPath + subPathOrNewFileName + (subPathOrNewFileName.endsWith("/") ? "" : "/");
+ sourceFilePath = s3pathAndSourceFile[1];
+ uploadedFileName = FilenameUtils.getName(s3pathAndSourceFile[1]);
+ }
+ } else if (size == 3) {
+ path = definitionPath + s3pathAndSourceFile[0] + (s3pathAndSourceFile[0].endsWith("/") ? "" : "/");
+ uploadedFileName = s3pathAndSourceFile[1];
+ sourceFilePath = s3pathAndSourceFile[2];
+
+ } else {
+ throw new IllegalArgumentException();
+ }
+
+ String keyPath = path + uploadedFileName;
+ int k = 1;
+ while (fileNames.contains(keyPath)) {
+ keyPath = path + (k++) + uploadedFileName;
+ }
+ fileNames.add(keyPath);
+ client.putObject(PUT_OBJECT_BUILDER.bucket(bucketName).key(keyPath).build(),
+ RequestBody.fromFile(Paths.get(sourceFilePath)));
+ }
+ LOGGER.info("Done creating bucket with data");
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/WildCardToRegexTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/WildCardToRegexTest.java
new file mode 100644
index 0000000..f15b657
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/external_dataset/aws/WildCardToRegexTest.java
@@ -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.
+ */
+
+package org.apache.asterix.test.external_dataset.aws;
+
+import static org.apache.asterix.external.util.ExternalDataUtils.patternToRegex;
+
+import java.util.regex.Pattern;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.junit.Test;
+
+import junit.framework.TestCase;
+
+public class WildCardToRegexTest extends TestCase {
+
+ @Test
+ public void test() throws HyracksDataException {
+ String result = patternToRegex("*?[abc]");
+ assertEquals(".*.[abc]", result);
+ Pattern.compile(result);
+
+ result = patternToRegex("*?[!@#$%^&*()+<>|=!{}.]");
+ assertEquals(".*.[^@#\\$%\\^&\\*\\(\\)\\+\\<\\>\\|\\=\\!\\{\\}\\.]", result);
+ Pattern.compile(result);
+
+ result = patternToRegex("**??[[a-z*0-9]]");
+ assertEquals(".*.*..[\\[a-z\\*0-9]\\]", result);
+ Pattern.compile(result);
+
+ result = patternToRegex("**??[[a-z*0-9]]");
+ assertEquals(".*.*..[\\[a-z\\*0-9]\\]", result);
+ Pattern.compile(result);
+
+ result = patternToRegex("*?[!abc]");
+ assertEquals(".*.[^abc]", result);
+ Pattern.compile(result);
+
+ result = patternToRegex("[!]abc");
+ assertEquals("\\[\\!\\]abc", result);
+ Pattern.compile(result);
+
+ result = patternToRegex("[!]abc]");
+ assertEquals("[^\\]abc]", result);
+ Pattern.compile(result);
+
+ result = patternToRegex("[]]");
+ assertEquals("[\\]]", result);
+ Pattern.compile(result);
+
+ result = patternToRegex("[]abcd");
+ assertEquals("\\[\\]abcd", result);
+ Pattern.compile(result);
+
+ result = patternToRegex("[]abcd]");
+ assertEquals("[\\]abcd]", result);
+ Pattern.compile(result);
+
+ result = patternToRegex("[^]");
+ assertEquals("[\\^]", result);
+ Pattern.compile(result);
+
+ result = patternToRegex("[^]]");
+ assertEquals("[\\^]\\]", result);
+ Pattern.compile(result);
+
+ result = patternToRegex("[!]");
+ assertEquals("\\[\\!\\]", result);
+ Pattern.compile(result);
+
+ result = patternToRegex("[!]]");
+ assertEquals("[^\\]]", result);
+ Pattern.compile(result);
+
+ result = patternToRegex("[][!][^]]]]*[![*a-zA--&&^$||0-9B$\\\\*&&]*&&[^a-b||0--9][[[");
+ assertEquals(
+ "[\\]\\[\\!][\\^]\\]\\]\\].*[^\\[\\*a-zA\\-\\-\\&\\&\\^\\$\\|\\|0-9B\\$\\\\\\\\\\*\\&\\&].*&&[\\^a-b\\|\\|0\\-\\-9]\\[\\[\\[",
+ result);
+ Pattern.compile(result);
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
index 6e0413c..c3dc821 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
@@ -18,17 +18,19 @@
*/
package org.apache.asterix.test.optimizer;
-import java.io.BufferedReader;
import java.io.File;
-import java.io.FileInputStream;
-import java.io.InputStreamReader;
import java.io.PrintWriter;
import java.io.StringReader;
import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
import org.apache.asterix.api.java.AsterixJavaClient;
@@ -93,6 +95,9 @@
protected static AsterixHyracksIntegrationUtil integrationUtil = new AsterixHyracksIntegrationUtil();
+ private static final String PATTERN_VAR_ID_PREFIX = "\\$\\$";
+ private static final Pattern PATTERN_VAR_ID = Pattern.compile(PATTERN_VAR_ID_PREFIX + "(\\d+)");
+
@BeforeClass
public static void setUp() throws Exception {
final File outdir = new File(PATH_ACTUAL);
@@ -205,37 +210,36 @@
throw new Exception("Compile ERROR for " + queryFile + ": " + e.getMessage(), e);
}
- BufferedReader readerExpected =
- new BufferedReader(new InputStreamReader(new FileInputStream(expectedFile), "UTF-8"));
- BufferedReader readerActual =
- new BufferedReader(new InputStreamReader(new FileInputStream(actualFile), "UTF-8"));
+ List<String> linesExpected = Files.readAllLines(expectedFile.toPath(), StandardCharsets.UTF_8);
+ List<String> linesActual = Files.readAllLines(actualFile.toPath(), StandardCharsets.UTF_8);
+ int varBaseExpected = findBaseVarId(linesExpected);
+ int varBaseActual = findBaseVarId(linesActual);
+
+ Iterator<String> readerExpected = linesExpected.iterator();
+ Iterator<String> readerActual = linesActual.iterator();
String lineExpected, lineActual;
int num = 1;
- try {
- while ((lineExpected = readerExpected.readLine()) != null) {
- lineActual = readerActual.readLine();
- if (lineActual == null) {
- throw new Exception("Result for " + queryFile + " changed at line " + num + ":\n< "
- + lineExpected + "\n> ");
- }
- if (!lineExpected.equals(lineActual)) {
- throw new Exception("Result for " + queryFile + " changed at line " + num + ":\n< "
- + lineExpected + "\n> " + lineActual);
- }
- ++num;
- }
- lineActual = readerActual.readLine();
- if (lineActual != null) {
+ while (readerExpected.hasNext()) {
+ lineExpected = readerExpected.next();
+ if (!readerActual.hasNext()) {
throw new Exception(
- "Result for " + queryFile + " changed at line " + num + ":\n< \n> " + lineActual);
+ "Result for " + queryFile + " changed at line " + num + ":\n< " + lineExpected + "\n> ");
}
- LOGGER.info("Test \"" + queryFile.getPath() + "\" PASSED!");
- actualFile.delete();
- } finally {
- readerExpected.close();
- readerActual.close();
+ lineActual = readerActual.next();
+
+ if (!planLineEquals(lineExpected, varBaseExpected, lineActual, varBaseActual)) {
+ throw new Exception("Result for " + queryFile + " changed at line " + num + ":\n< " + lineExpected
+ + "\n> " + lineActual);
+ }
+ ++num;
}
+ if (readerActual.hasNext()) {
+ throw new Exception(
+ "Result for " + queryFile + " changed at line " + num + ":\n< \n> " + readerActual.next());
+ }
+ LOGGER.info("Test \"" + queryFile.getPath() + "\" PASSED!");
+ actualFile.delete();
} catch (Exception e) {
if (!(e instanceof AssumptionViolatedException)) {
LOGGER.error("Test \"" + queryFile.getPath() + "\" FAILED!");
@@ -245,4 +249,40 @@
}
}
}
+
+ private boolean planLineEquals(String lineExpected, int varIdBaseExpected, String lineActual, int varIdBaseActual) {
+ String lineExpectedNorm = normalizePlanLine(lineExpected, varIdBaseExpected);
+ String lineActualNorm = normalizePlanLine(lineActual, varIdBaseActual);
+ return lineExpectedNorm.equals(lineActualNorm);
+ }
+
+ // rewrite variable ids in given plan line: $$varId -> $$(varId-varIdBase)
+ private String normalizePlanLine(String line, int varIdBase) {
+ if (varIdBase == Integer.MAX_VALUE) {
+ // plan did not contain any variables -> no rewriting necessary
+ return line;
+ }
+ Matcher m = PATTERN_VAR_ID.matcher(line);
+ StringBuffer sb = new StringBuffer(line.length());
+ while (m.find()) {
+ int varId = Integer.parseInt(m.group(1));
+ int newVarId = varId - varIdBase;
+ m.appendReplacement(sb, PATTERN_VAR_ID_PREFIX + newVarId);
+ }
+ m.appendTail(sb);
+ return sb.toString();
+ }
+
+ private int findBaseVarId(Collection<String> plan) {
+ int varIdBase = Integer.MAX_VALUE;
+ Matcher m = PATTERN_VAR_ID.matcher("");
+ for (String line : plan) {
+ m.reset(line);
+ while (m.find()) {
+ int varId = Integer.parseInt(m.group(1));
+ varIdBase = Math.min(varIdBase, varId);
+ }
+ }
+ return varIdBase;
+ }
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
index 4ed36c6..30ebc1d 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
@@ -19,7 +19,6 @@
package org.apache.asterix.test.runtime;
import java.io.File;
-import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.List;
@@ -34,6 +33,7 @@
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hyracks.api.config.IOption;
import org.apache.hyracks.control.nc.NodeControllerService;
+import org.apache.hyracks.util.ThrowingSupplier;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -123,8 +123,12 @@
HDFSCluster.getInstance().cleanup();
}
+ saveFailedXml(() -> File.createTempFile("failed", ".xml"));
+ }
+
+ public static void saveFailedXml(ThrowingSupplier<File> destFileSupplier) throws Exception {
if (FailedGroup != null && FailedGroup.getTestCase().size() > 0) {
- File temp = File.createTempFile("failed", ".xml");
+ File failedFile = destFileSupplier.get();
javax.xml.bind.JAXBContext jaxbCtx = null;
jaxbCtx = javax.xml.bind.JAXBContext.newInstance(TestSuite.class.getPackage().getName());
javax.xml.bind.Marshaller marshaller = null;
@@ -133,11 +137,12 @@
TestSuite failedSuite = new TestSuite();
failedSuite.setResultOffsetPath("results");
failedSuite.setQueryOffsetPath("queries");
+ failedSuite.setQueryFileExtension(".sqlpp");
failedSuite.getTestGroup().add(FailedGroup);
- marshaller.marshal(failedSuite, temp);
- System.err.println("The failed.xml is written to :" + temp.getAbsolutePath()
- + ". You can copy it to only.xml by the following cmd:" + "\rcp " + temp.getAbsolutePath() + " "
- + Paths.get("./src/test/resources/runtimets/only.xml").toAbsolutePath());
+ marshaller.marshal(failedSuite, failedFile);
+ LOGGER.error("#####");
+ LOGGER.error("##### the failed.xml is written to {}", failedFile.getAbsolutePath());
+ LOGGER.error("#####");
}
}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ResultStreamingFailureTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ResultStreamingFailureTest.java
new file mode 100644
index 0000000..c4fca1a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ResultStreamingFailureTest.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.runtime;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
+import org.apache.http.NameValuePair;
+import org.apache.http.client.entity.UrlEncodedFormEntity;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+import org.apache.http.message.BasicNameValuePair;
+import org.apache.hyracks.control.nc.result.ResultPartitionReader;
+import org.apache.hyracks.util.Span;
+import org.apache.hyracks.util.ThreadDumpUtil;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class ResultStreamingFailureTest {
+
+ private static final AsterixHyracksIntegrationUtil integrationUtil = new AsterixHyracksIntegrationUtil();
+
+ @Before
+ public void setUp() throws Exception {
+ integrationUtil.init(true, AsterixHyracksIntegrationUtil.DEFAULT_CONF_FILE);
+ }
+
+ @After
+ public void tearDown() throws Exception {
+ integrationUtil.deinit(true);
+ }
+
+ @Test
+ public void resultStreamingFailureTest() throws Exception {
+ queryAndDropConnection();
+ // allow result sender to terminate and ensure no leaks
+ Span timeout = Span.start(30, TimeUnit.SECONDS);
+ while (!timeout.elapsed()) {
+ String threadDump = ThreadDumpUtil.takeDumpString();
+ if (!threadDump.contains(ResultPartitionReader.class.getName())) {
+ return;
+ }
+ TimeUnit.SECONDS.sleep(1);
+ }
+ throw new AssertionError("found leaking senders in:\n" + ThreadDumpUtil.takeDumpString());
+ }
+
+ private void queryAndDropConnection() throws IOException {
+ try (CloseableHttpClient httpClient = HttpClients.createDefault();) {
+ final List<NameValuePair> params = new ArrayList<>();
+ params.add(new BasicNameValuePair("statement", "select * from range(1, 10000000) r;"));
+ HttpPost request = new HttpPost("http://localhost:19004/query/service");
+ request.setEntity(new UrlEncodedFormEntity(params, StandardCharsets.UTF_8));
+ CloseableHttpResponse response = httpClient.execute(request);
+ Assert.assertEquals(200, response.getStatusLine().getStatusCode());
+ // close connection without streaming the result
+ }
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
index c71cf31..53201f5 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
@@ -19,7 +19,6 @@
package org.apache.asterix.test.sqlpp;
import static org.apache.hyracks.util.file.FileUtil.canonicalize;
-import static org.mockito.Matchers.anyString;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
@@ -58,6 +57,7 @@
import org.apache.asterix.testframework.xml.TestGroup;
import org.apache.hyracks.test.support.TestUtils;
import org.junit.Assert;
+import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
@@ -136,16 +136,17 @@
when(metadataProvider.getDefaultDataverseName()).thenReturn(dvName);
when(metadataProvider.getConfig()).thenReturn(config);
when(config.get(FunctionUtil.IMPORT_PRIVATE_FUNCTIONS)).thenReturn("true");
- when(metadataProvider.findDataset(anyString(), anyString())).thenAnswer(new Answer<Dataset>() {
- @Override
- public Dataset answer(InvocationOnMock invocation) {
- Object[] args = invocation.getArguments();
- final Dataset mockDataset = mock(Dataset.class);
- String fullyQualifiedName = args[0] != null ? args[0] + "." + args[1] : (String) args[1];
- when(mockDataset.getFullyQualifiedName()).thenReturn(fullyQualifiedName);
- return mockDataset;
- }
- });
+ when(metadataProvider.findDataset(Mockito.<String> any(), Mockito.<String> any()))
+ .thenAnswer(new Answer<Dataset>() {
+ @Override
+ public Dataset answer(InvocationOnMock invocation) {
+ Object[] args = invocation.getArguments();
+ final Dataset mockDataset = mock(Dataset.class);
+ String fullyQualifiedName = args[0] != null ? args[0] + "." + args[1] : (String) args[1];
+ when(mockDataset.getFullyQualifiedName()).thenReturn(fullyQualifiedName);
+ return mockDataset;
+ }
+ });
for (Statement st : statements) {
if (st.getKind() == Statement.Kind.QUERY) {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/IndexDropOperatorNodePushableTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/IndexDropOperatorNodePushableTest.java
index 31a6004..8cce055 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/IndexDropOperatorNodePushableTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/storage/IndexDropOperatorNodePushableTest.java
@@ -178,7 +178,7 @@
dropInUseOp.initialize();
} catch (HyracksDataException e) {
e.printStackTrace();
- Assert.assertEquals(ErrorCode.CANNOT_DROP_IN_USE_INDEX, e.getErrorCode());
+ Assert.assertTrue(e.matches(ErrorCode.CANNOT_DROP_IN_USE_INDEX));
dropFailed.set(true);
}
Assert.assertTrue(dropFailed.get());
@@ -218,7 +218,7 @@
dropNonExistingOp.initialize();
} catch (HyracksDataException e) {
e.printStackTrace();
- Assert.assertEquals(ErrorCode.INDEX_DOES_NOT_EXIST, e.getErrorCode());
+ Assert.assertTrue(e.matches(ErrorCode.INDEX_DOES_NOT_EXIST));
dropFailed.set(true);
}
Assert.assertTrue(dropFailed.get());
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/builtin_type/meta03_builtin_type.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/builtin_type/meta03_builtin_type.1.adm
index 9427035..63db00f 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/builtin_type/meta03_builtin_type.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/builtin_type/meta03_builtin_type.1.adm
@@ -1 +1 @@
-{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Mon Sep 17 23:18:30 PDT 2012" }
+{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Mon Sep 17 23:18:30 PDT 2012" }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/builtin_type_nullable/meta03_builtin_type_nullable.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/builtin_type_nullable/meta03_builtin_type_nullable.1.adm
index 2abeba4..97efb63 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/builtin_type_nullable/meta03_builtin_type_nullable.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/builtin_type_nullable/meta03_builtin_type_nullable.1.adm
@@ -1 +1 @@
-{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": true } ] } }, "Timestamp": "Mon Sep 17 23:18:30 PDT 2012" }
+{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": true, "IsMissable": true } ] } }, "Timestamp": "Mon Sep 17 23:18:30 PDT 2012" }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/complex_type/ordered_list/meta03_ordered_list.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/complex_type/ordered_list/meta03_ordered_list.1.adm
index 25d210d..66558fd 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/complex_type/ordered_list/meta03_ordered_list.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/complex_type/ordered_list/meta03_ordered_list.1.adm
@@ -1 +1 @@
-{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false }, { "FieldName": "list", "FieldType": "testtype_list", "IsNullable": false } ] } }, "Timestamp": "Thu Feb 15 19:45:50 PST 2018" }
+{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "list", "FieldType": "testtype_list", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Thu Feb 15 19:45:50 PST 2018" }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/complex_type/ordered_list_nullable/meta03_ordered_list_nullable.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/complex_type/ordered_list_nullable/meta03_ordered_list_nullable.1.adm
index 0ee413e..e6afb76 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/complex_type/ordered_list_nullable/meta03_ordered_list_nullable.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/complex_type/ordered_list_nullable/meta03_ordered_list_nullable.1.adm
@@ -1 +1 @@
-{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false }, { "FieldName": "list", "FieldType": "testtype_list", "IsNullable": true } ] } }, "Timestamp": "Thu Feb 15 19:45:50 PST 2018" }
+{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "list", "FieldType": "testtype_list", "IsNullable": true, "IsMissable": true } ] } }, "Timestamp": "Thu Feb 15 19:45:50 PST 2018" }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/complex_type/record/meta03_record.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/complex_type/record/meta03_record.1.adm
index feb010d..831d735 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/complex_type/record/meta03_record.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/complex_type/record/meta03_record.1.adm
@@ -1 +1 @@
-{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false }, { "FieldName": "subtype", "FieldType": "testtype_subtype", "IsNullable": false } ] } }, "Timestamp": "Thu Feb 15 19:45:50 PST 2018" }
+{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "subtype", "FieldType": "testtype_subtype", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Thu Feb 15 19:45:50 PST 2018" }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/complex_type/record_nullable/meta03_record_nullable.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/complex_type/record_nullable/meta03_record_nullable.1.adm
index eb19c4e..4479d4c 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/complex_type/record_nullable/meta03_record_nullable.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/complex_type/record_nullable/meta03_record_nullable.1.adm
@@ -1 +1 @@
-{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false }, { "FieldName": "subtype", "FieldType": "testtype_subtype", "IsNullable": true } ] } }, "Timestamp": "Thu Feb 15 19:45:50 PST 2018" }
+{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "subtype", "FieldType": "testtype_subtype", "IsNullable": true, "IsMissable": true } ] } }, "Timestamp": "Thu Feb 15 19:45:50 PST 2018" }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/complex_type/unordered_list/meta03_unordered_list.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/complex_type/unordered_list/meta03_unordered_list.1.adm
index 25d210d..66558fd 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/complex_type/unordered_list/meta03_unordered_list.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/complex_type/unordered_list/meta03_unordered_list.1.adm
@@ -1 +1 @@
-{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false }, { "FieldName": "list", "FieldType": "testtype_list", "IsNullable": false } ] } }, "Timestamp": "Thu Feb 15 19:45:50 PST 2018" }
+{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "list", "FieldType": "testtype_list", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Thu Feb 15 19:45:50 PST 2018" }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/complex_type/unordered_list_nullable/meta03_unordered_list_nullable.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/complex_type/unordered_list_nullable/meta03_unordered_list_nullable.1.adm
index 0ee413e..e6afb76 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/complex_type/unordered_list_nullable/meta03_unordered_list_nullable.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta03/complex_type/unordered_list_nullable/meta03_unordered_list_nullable.1.adm
@@ -1 +1 @@
-{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false }, { "FieldName": "list", "FieldType": "testtype_list", "IsNullable": true } ] } }, "Timestamp": "Thu Feb 15 19:45:50 PST 2018" }
+{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "list", "FieldType": "testtype_list", "IsNullable": true, "IsMissable": true } ] } }, "Timestamp": "Thu Feb 15 19:45:50 PST 2018" }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/builtin_type/meta04_builtin_type.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/builtin_type/meta04_builtin_type.1.adm
index b28b3c8..0cf1897 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/builtin_type/meta04_builtin_type.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/builtin_type/meta04_builtin_type.1.adm
@@ -1 +1 @@
-{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Thu Feb 15 19:45:50 PST 2018" }
+{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Thu Feb 15 19:45:50 PST 2018" }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/builtin_type_nullable/meta04_builtin_type_nullable.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/builtin_type_nullable/meta04_builtin_type_nullable.1.adm
index e67cbd4..4428bc0 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/builtin_type_nullable/meta04_builtin_type_nullable.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/builtin_type_nullable/meta04_builtin_type_nullable.1.adm
@@ -1 +1 @@
-{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": true } ] } }, "Timestamp": "Thu Feb 15 19:45:50 PST 2018" }
+{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": true, "IsMissable": true } ] } }, "Timestamp": "Thu Feb 15 19:45:50 PST 2018" }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/complex_type/ordered_list/meta04_ordered_list.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/complex_type/ordered_list/meta04_ordered_list.1.adm
index 5c98d74..badf77f 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/complex_type/ordered_list/meta04_ordered_list.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/complex_type/ordered_list/meta04_ordered_list.1.adm
@@ -1 +1 @@
-{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false }, { "FieldName": "list", "FieldType": "testtype_list", "IsNullable": false } ] } }, "Timestamp": "Thu Feb 15 19:45:51 PST 2018" }
+{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "list", "FieldType": "testtype_list", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Thu Feb 15 19:45:51 PST 2018" }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/complex_type/ordered_list_nullable/meta04_ordered_list_nullable.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/complex_type/ordered_list_nullable/meta04_ordered_list_nullable.1.adm
index 697f652..d13ba1d 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/complex_type/ordered_list_nullable/meta04_ordered_list_nullable.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/complex_type/ordered_list_nullable/meta04_ordered_list_nullable.1.adm
@@ -1 +1 @@
-{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false }, { "FieldName": "list", "FieldType": "testtype_list", "IsNullable": true } ] } }, "Timestamp": "Thu Feb 15 19:45:51 PST 2018" }
+{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "list", "FieldType": "testtype_list", "IsNullable": true, "IsMissable": true } ] } }, "Timestamp": "Thu Feb 15 19:45:51 PST 2018" }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/complex_type/record/meta04_record.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/complex_type/record/meta04_record.1.adm
index 620976f..e73474e 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/complex_type/record/meta04_record.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/complex_type/record/meta04_record.1.adm
@@ -1 +1 @@
-{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false }, { "FieldName": "subtype", "FieldType": "testtype_subtype", "IsNullable": false } ] } }, "Timestamp": "Thu Feb 15 19:45:51 PST 2018" }
+{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "subtype", "FieldType": "testtype_subtype", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Thu Feb 15 19:45:51 PST 2018" }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/complex_type/record_nullable/meta04_record_nullable.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/complex_type/record_nullable/meta04_record_nullable.1.adm
index e95590d..218f386 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/complex_type/record_nullable/meta04_record_nullable.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/complex_type/record_nullable/meta04_record_nullable.1.adm
@@ -1 +1 @@
-{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false }, { "FieldName": "subtype", "FieldType": "testtype_subtype", "IsNullable": true } ] } }, "Timestamp": "Thu Feb 15 19:45:51 PST 2018" }
+{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "subtype", "FieldType": "testtype_subtype", "IsNullable": true, "IsMissable": true } ] } }, "Timestamp": "Thu Feb 15 19:45:51 PST 2018" }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/complex_type/unordered_list/meta04_unordered_list.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/complex_type/unordered_list/meta04_unordered_list.1.adm
index 5c98d74..badf77f 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/complex_type/unordered_list/meta04_unordered_list.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/complex_type/unordered_list/meta04_unordered_list.1.adm
@@ -1 +1 @@
-{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false }, { "FieldName": "list", "FieldType": "testtype_list", "IsNullable": false } ] } }, "Timestamp": "Thu Feb 15 19:45:51 PST 2018" }
+{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "list", "FieldType": "testtype_list", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Thu Feb 15 19:45:51 PST 2018" }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/complex_type/unordered_list_nullable/meta04_unordered_list_nullable.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/complex_type/unordered_list_nullable/meta04_unordered_list_nullable.1.adm
index 697f652..d13ba1d 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/complex_type/unordered_list_nullable/meta04_unordered_list_nullable.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/meta04/complex_type/unordered_list_nullable/meta04_unordered_list_nullable.1.adm
@@ -1 +1 @@
-{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false }, { "FieldName": "list", "FieldType": "testtype_list", "IsNullable": true } ] } }, "Timestamp": "Thu Feb 15 19:45:51 PST 2018" }
+{ "DataverseName": "testdv", "DatatypeName": "testtype", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "list", "FieldType": "testtype_list", "IsNullable": true, "IsMissable": true } ] } }, "Timestamp": "Thu Feb 15 19:45:51 PST 2018" }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
index 5fc2c1e..5755493 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
@@ -1,70 +1,70 @@
-{ "DataverseName": "Metadata", "DatatypeName": "AnyObject", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ ] } }, "Timestamp": "Mon Jan 08 10:27:05 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "CompactionPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "CompactionPolicy", "FieldType": "string", "IsNullable": false }, { "FieldName": "Classname", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatatypeDataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatatypeName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetType", "FieldType": "string", "IsNullable": false }, { "FieldName": "GroupName", "FieldType": "string", "IsNullable": false }, { "FieldName": "CompactionPolicy", "FieldType": "string", "IsNullable": false }, { "FieldName": "CompactionPolicyProperties", "FieldType": "DatasetRecordType_CompactionPolicyProperties", "IsNullable": false }, { "FieldName": "InternalDetails", "FieldType": "DatasetRecordType_InternalDetails", "IsNullable": true }, { "FieldName": "ExternalDetails", "FieldType": "DatasetRecordType_ExternalDetails", "IsNullable": true }, { "FieldName": "Hints", "FieldType": "DatasetRecordType_Hints", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetId", "FieldType": "int32", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_CompactionPolicyProperties", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_CompactionPolicyProperties_Item" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_CompactionPolicyProperties_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_ExternalDetails", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string", "IsNullable": false }, { "FieldName": "Properties", "FieldType": "DatasetRecordType_ExternalDetails_Properties", "IsNullable": false }, { "FieldName": "LastRefreshTime", "FieldType": "datetime", "IsNullable": false }, { "FieldName": "TransactionState", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_ExternalDetails_Properties", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_ExternalDetails_Properties_Item" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_ExternalDetails_Properties_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_Hints", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "DatasetRecordType_Hints_Item" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_Hints_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string", "IsNullable": false }, { "FieldName": "PartitioningStrategy", "FieldType": "string", "IsNullable": false }, { "FieldName": "PartitioningKey", "FieldType": "DatasetRecordType_InternalDetails_PartitioningKey", "IsNullable": false }, { "FieldName": "PrimaryKey", "FieldType": "DatasetRecordType_InternalDetails_PrimaryKey", "IsNullable": false }, { "FieldName": "Autogenerated", "FieldType": "boolean", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PartitioningKey", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_InternalDetails_PartitioningKey_Item" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PartitioningKey_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PrimaryKey", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_InternalDetails_PrimaryKey_Item" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PrimaryKey_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Classname", "FieldType": "string", "IsNullable": false }, { "FieldName": "Type", "FieldType": "string", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatatypeName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Derived", "FieldType": "DatatypeRecordType_Derived", "IsNullable": true }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string", "IsNullable": false }, { "FieldName": "IsAnonymous", "FieldType": "boolean", "IsNullable": false }, { "FieldName": "Record", "FieldType": "DatatypeRecordType_Derived_Record", "IsNullable": true }, { "FieldName": "UnorderedList", "FieldType": "string", "IsNullable": true }, { "FieldName": "OrderedList", "FieldType": "string", "IsNullable": true } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived_Record", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean", "IsNullable": false }, { "FieldName": "Fields", "FieldType": "DatatypeRecordType_Derived_Record_Fields", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived_Record_Fields", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatatypeRecordType_Derived_Record_Fields_Item" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived_Record_Fields_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FieldType", "FieldType": "string", "IsNullable": false }, { "FieldName": "IsNullable", "FieldType": "boolean", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DataFormat", "FieldType": "string", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "ExternalFileRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FileNumber", "FieldType": "int32", "IsNullable": false }, { "FieldName": "FileName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FileSize", "FieldType": "int64", "IsNullable": false }, { "FieldName": "FileModTime", "FieldType": "datetime", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedConnectionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FeedName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false }, { "FieldName": "ReturnType", "FieldType": "string", "IsNullable": false }, { "FieldName": "AppliedFunctions", "FieldType": "FeedConnectionRecordType_AppliedFunctions", "IsNullable": false }, { "FieldName": "PolicyName", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedConnectionRecordType_AppliedFunctions", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "string" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "PolicyName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Description", "FieldType": "string", "IsNullable": false }, { "FieldName": "Properties", "FieldType": "FeedPolicyRecordType_Properties", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType_Properties", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "FeedPolicyRecordType_Properties_Item" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType_Properties_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "FeedName", "FieldType": "string", "IsNullable": false }, { "FieldName": "AdapterConfiguration", "FieldType": "FeedRecordType_AdapterConfiguration", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType_AdapterConfiguration", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "FeedRecordType_AdapterConfiguration_Item" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType_AdapterConfiguration_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Arity", "FieldType": "string", "IsNullable": false }, { "FieldName": "Params", "FieldType": "FunctionRecordType_Params", "IsNullable": false }, { "FieldName": "ReturnType", "FieldType": "string", "IsNullable": false }, { "FieldName": "Definition", "FieldType": "string", "IsNullable": false }, { "FieldName": "Language", "FieldType": "string", "IsNullable": false }, { "FieldName": "Kind", "FieldType": "string", "IsNullable": false }, { "FieldName": "Dependencies", "FieldType": "FunctionRecordType_Dependencies", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType_Dependencies", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "FunctionRecordType_Dependencies_Item" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType_Dependencies_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "FunctionRecordType_Dependencies_Item_Item" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType_Dependencies_Item_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType_Params", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false }, { "FieldName": "IndexName", "FieldType": "string", "IsNullable": false }, { "FieldName": "IndexStructure", "FieldType": "string", "IsNullable": false }, { "FieldName": "SearchKey", "FieldType": "IndexRecordType_SearchKey", "IsNullable": false }, { "FieldName": "IsPrimary", "FieldType": "boolean", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType_SearchKey", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "IndexRecordType_SearchKey_Item" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType_SearchKey_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false }, { "FieldName": "Name", "FieldType": "string", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string", "IsNullable": false }, { "FieldName": "NodeNames", "FieldType": "NodeGroupRecordType_NodeNames", "IsNullable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType_NodeNames", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "string" }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string", "IsNullable": false }, { "FieldName": "NumberOfCores", "FieldType": "int64", "IsNullable": false }, { "FieldName": "WorkingMemorySize", "FieldType": "int64", "IsNullable": false } ] } }, "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "binary", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "circle", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "date", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "double", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "duration", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "float", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "geometry", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "int16", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "int32", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "int64", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "int8", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "interval", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "line", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "missing", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "null", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "point", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "shortwithouttypeinfo", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "string", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "time", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "uuid", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
-{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Timestamp": "Mon Jan 08 10:27:04 PST 2018" }
\ No newline at end of file
+{ "DataverseName": "Metadata", "DatatypeName": "AnyObject", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ ] } }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "CompactionPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "CompactionPolicy", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Classname", "FieldType": "string", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "DatatypeDataverseName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "DatatypeName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "DatasetType", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "GroupName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "CompactionPolicy", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "CompactionPolicyProperties", "FieldType": "DatasetRecordType_CompactionPolicyProperties", "IsNullable": false, "IsMissable": false }, { "FieldName": "InternalDetails", "FieldType": "DatasetRecordType_InternalDetails", "IsNullable": true, "IsMissable": true }, { "FieldName": "ExternalDetails", "FieldType": "DatasetRecordType_ExternalDetails", "IsNullable": true, "IsMissable": true }, { "FieldName": "Hints", "FieldType": "DatasetRecordType_Hints", "IsNullable": false, "IsMissable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "DatasetId", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_CompactionPolicyProperties", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_CompactionPolicyProperties_Item" }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_CompactionPolicyProperties_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_ExternalDetails", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Properties", "FieldType": "DatasetRecordType_ExternalDetails_Properties", "IsNullable": false, "IsMissable": false }, { "FieldName": "LastRefreshTime", "FieldType": "datetime", "IsNullable": false, "IsMissable": false }, { "FieldName": "TransactionState", "FieldType": "int32", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_ExternalDetails_Properties", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_ExternalDetails_Properties_Item" }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_ExternalDetails_Properties_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_Hints", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "DatasetRecordType_Hints_Item" }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_Hints_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "PartitioningStrategy", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "PartitioningKey", "FieldType": "DatasetRecordType_InternalDetails_PartitioningKey", "IsNullable": false, "IsMissable": false }, { "FieldName": "PrimaryKey", "FieldType": "DatasetRecordType_InternalDetails_PrimaryKey", "IsNullable": false, "IsMissable": false }, { "FieldName": "Autogenerated", "FieldType": "boolean", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PartitioningKey", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_InternalDetails_PartitioningKey_Item" }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PartitioningKey_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PrimaryKey", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatasetRecordType_InternalDetails_PrimaryKey_Item" }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType_InternalDetails_PrimaryKey_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Name", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Classname", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Type", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "DatatypeName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Derived", "FieldType": "DatatypeRecordType_Derived", "IsNullable": true, "IsMissable": true }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "IsAnonymous", "FieldType": "boolean", "IsNullable": false, "IsMissable": false }, { "FieldName": "Record", "FieldType": "DatatypeRecordType_Derived_Record", "IsNullable": true, "IsMissable": true }, { "FieldName": "UnorderedList", "FieldType": "string", "IsNullable": true, "IsMissable": true }, { "FieldName": "OrderedList", "FieldType": "string", "IsNullable": true, "IsMissable": true } ] } }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived_Record", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean", "IsNullable": false, "IsMissable": false }, { "FieldName": "Fields", "FieldType": "DatatypeRecordType_Derived_Record_Fields", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived_Record_Fields", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "DatatypeRecordType_Derived_Record_Fields_Item" }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType_Derived_Record_Fields_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "FieldType", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "IsNullable", "FieldType": "boolean", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "DataFormat", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "ExternalFileRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "FileNumber", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "FileName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "FileSize", "FieldType": "int64", "IsNullable": false, "IsMissable": false }, { "FieldName": "FileModTime", "FieldType": "datetime", "IsNullable": false, "IsMissable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedConnectionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "FeedName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "ReturnType", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "AppliedFunctions", "FieldType": "FeedConnectionRecordType_AppliedFunctions", "IsNullable": false, "IsMissable": false }, { "FieldName": "PolicyName", "FieldType": "string", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedConnectionRecordType_AppliedFunctions", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "string" }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "PolicyName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Description", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Properties", "FieldType": "FeedPolicyRecordType_Properties", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType_Properties", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "FeedPolicyRecordType_Properties_Item" }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedPolicyRecordType_Properties_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "FeedName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "AdapterConfiguration", "FieldType": "FeedRecordType_AdapterConfiguration", "IsNullable": false, "IsMissable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType_AdapterConfiguration", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "FeedRecordType_AdapterConfiguration_Item" }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "FeedRecordType_AdapterConfiguration_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Value", "FieldType": "string", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Name", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Arity", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Params", "FieldType": "FunctionRecordType_Params", "IsNullable": false, "IsMissable": false }, { "FieldName": "ReturnType", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Definition", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Language", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Kind", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Dependencies", "FieldType": "FunctionRecordType_Dependencies", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType_Dependencies", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "FunctionRecordType_Dependencies_Item" }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType_Dependencies_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "FunctionRecordType_Dependencies_Item_Item" }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType_Dependencies_Item_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType_Params", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "DatasetName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "IndexName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "IndexStructure", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "SearchKey", "FieldType": "IndexRecordType_SearchKey", "IsNullable": false, "IsMissable": false }, { "FieldName": "IsPrimary", "FieldType": "boolean", "IsNullable": false, "IsMissable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "PendingOp", "FieldType": "int32", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType_SearchKey", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "IndexRecordType_SearchKey_Item" }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType_SearchKey_Item", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "string" }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "LibraryRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Name", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "NodeNames", "FieldType": "NodeGroupRecordType_NodeNames", "IsNullable": false, "IsMissable": false }, { "FieldName": "Timestamp", "FieldType": "string", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType_NodeNames", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "string" }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "NumberOfCores", "FieldType": "int64", "IsNullable": false, "IsMissable": false }, { "FieldName": "WorkingMemorySize", "FieldType": "int64", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "binary", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "circle", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "date", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "double", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "duration", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "float", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "geometry", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "int16", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "int32", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "int64", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "int8", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "interval", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "line", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "missing", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "null", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "point", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "shortwithouttypeinfo", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "string", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "time", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "uuid", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Timestamp": "Fri May 08 17:24:56 PDT 2020" }
diff --git a/asterixdb/asterix-app/src/test/resources/metadata/results/transaction/verify_failure_previous_success/verify_failure_previous_success.1.adm b/asterixdb/asterix-app/src/test/resources/metadata/results/transaction/verify_failure_previous_success/verify_failure_previous_success.1.adm
index 4a0732b..ff88177 100644
--- a/asterixdb/asterix-app/src/test/resources/metadata/results/transaction/verify_failure_previous_success/verify_failure_previous_success.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/metadata/results/transaction/verify_failure_previous_success/verify_failure_previous_success.1.adm
@@ -1,10 +1,10 @@
-{ "DataverseName": "custord", "DatatypeName": "AddressType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "street", "FieldType": "StreetType", "IsNullable": false }, { "FieldName": "city", "FieldType": "string", "IsNullable": false }, { "FieldName": "state", "FieldType": "string", "IsNullable": false }, { "FieldName": "zip", "FieldType": "int16", "IsNullable": false } ] } }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
-{ "DataverseName": "custord", "DatatypeName": "CustomerType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "cid", "FieldType": "int32", "IsNullable": false }, { "FieldName": "name", "FieldType": "string", "IsNullable": false }, { "FieldName": "age", "FieldType": "int32", "IsNullable": true }, { "FieldName": "address", "FieldType": "AddressType", "IsNullable": true }, { "FieldName": "interests", "FieldType": "CustomerType_interests", "IsNullable": false }, { "FieldName": "children", "FieldType": "CustomerType_children", "IsNullable": false } ] } }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
-{ "DataverseName": "custord", "DatatypeName": "CustomerType_children", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "CustomerType_children_Item" }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
-{ "DataverseName": "custord", "DatatypeName": "CustomerType_children_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "name", "FieldType": "string", "IsNullable": false }, { "FieldName": "dob", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
-{ "DataverseName": "custord", "DatatypeName": "CustomerType_interests", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "string" }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
-{ "DataverseName": "custord", "DatatypeName": "OrderType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "oid", "FieldType": "int32", "IsNullable": false }, { "FieldName": "cid", "FieldType": "int32", "IsNullable": false }, { "FieldName": "orderstatus", "FieldType": "string", "IsNullable": false }, { "FieldName": "orderpriority", "FieldType": "string", "IsNullable": false }, { "FieldName": "clerk", "FieldType": "string", "IsNullable": false }, { "FieldName": "total", "FieldType": "float", "IsNullable": false }, { "FieldName": "items", "FieldType": "OrderType_items", "IsNullable": false } ] } }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
-{ "DataverseName": "custord", "DatatypeName": "OrderType_items", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "OrderType_items_Item" }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
-{ "DataverseName": "custord", "DatatypeName": "OrderType_items_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "number", "FieldType": "int64", "IsNullable": false }, { "FieldName": "storeIds", "FieldType": "OrderType_items_Item_storeIds", "IsNullable": false } ] } }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
-{ "DataverseName": "custord", "DatatypeName": "OrderType_items_Item_storeIds", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "int8" }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
-{ "DataverseName": "custord", "DatatypeName": "StreetType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "number", "FieldType": "int32", "IsNullable": true }, { "FieldName": "name", "FieldType": "string", "IsNullable": false } ] } }, "Timestamp": "Sat Nov 24 17:20:04 PST 2012" }
\ No newline at end of file
+{ "DataverseName": "custord", "DatatypeName": "AddressType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "street", "FieldType": "StreetType", "IsNullable": false, "IsMissable": false }, { "FieldName": "city", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "state", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "zip", "FieldType": "int16", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:25:04 PDT 2020" }
+{ "DataverseName": "custord", "DatatypeName": "CustomerType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "cid", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "name", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "age", "FieldType": "int32", "IsNullable": true, "IsMissable": true }, { "FieldName": "address", "FieldType": "AddressType", "IsNullable": true, "IsMissable": true }, { "FieldName": "interests", "FieldType": "CustomerType_interests", "IsNullable": false, "IsMissable": false }, { "FieldName": "children", "FieldType": "CustomerType_children", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:25:04 PDT 2020" }
+{ "DataverseName": "custord", "DatatypeName": "CustomerType_children", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "CustomerType_children_Item" }, "Timestamp": "Fri May 08 17:25:04 PDT 2020" }
+{ "DataverseName": "custord", "DatatypeName": "CustomerType_children_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "name", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "dob", "FieldType": "string", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:25:04 PDT 2020" }
+{ "DataverseName": "custord", "DatatypeName": "CustomerType_interests", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "string" }, "Timestamp": "Fri May 08 17:25:04 PDT 2020" }
+{ "DataverseName": "custord", "DatatypeName": "OrderType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "oid", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "cid", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "orderstatus", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "orderpriority", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "clerk", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "total", "FieldType": "float", "IsNullable": false, "IsMissable": false }, { "FieldName": "items", "FieldType": "OrderType_items", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:25:04 PDT 2020" }
+{ "DataverseName": "custord", "DatatypeName": "OrderType_items", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "OrderedList": "OrderType_items_Item" }, "Timestamp": "Fri May 08 17:25:04 PDT 2020" }
+{ "DataverseName": "custord", "DatatypeName": "OrderType_items_Item", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "number", "FieldType": "int64", "IsNullable": false, "IsMissable": false }, { "FieldName": "storeIds", "FieldType": "OrderType_items_Item_storeIds", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:25:04 PDT 2020" }
+{ "DataverseName": "custord", "DatatypeName": "OrderType_items_Item_storeIds", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "UnorderedList": "int8" }, "Timestamp": "Fri May 08 17:25:04 PDT 2020" }
+{ "DataverseName": "custord", "DatatypeName": "StreetType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "number", "FieldType": "int32", "IsNullable": true, "IsMissable": true }, { "FieldName": "name", "FieldType": "string", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri May 08 17:25:04 PDT 2020" }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/common-expr-01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/common-expr-01.sqlpp
new file mode 100644
index 0000000..30249d8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/common-expr-01.sqlpp
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Test common expression extraction
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+CREATE TYPE test.TestType AS
+{
+ id : INTEGER
+};
+
+CREATE DATASET t1(TestType) PRIMARY KEY id;
+
+WITH DT_PATTERN AS "YYYY-MM-DDThh:mm:ss"
+
+SELECT
+ v2.c11,
+ v1.c3,
+ SUM(v1.c4)
+FROM (
+ SELECT
+ c1,
+ c2,
+ GET_MONTH(PRINT_DATETIME(PARSE_DATETIME(c1,DT_PATTERN),DT_PATTERN)) AS c3,
+ (CASE WHEN CONTAINS(c2, "x") THEN 1 ELSE 0 END) AS c4
+ FROM test.t1
+) AS v1
+INNER JOIN (
+ SELECT
+ GET_HOUR(DURATION_FROM_INTERVAL(INTERVAL(
+ PARSE_DATETIME(PRINT_DATETIME(CURRENT_DATETIME(),DT_PATTERN),DT_PATTERN),
+ PARSE_DATETIME(MAX(PRINT_DATETIME(PARSE_DATETIME(t1.c5,DT_PATTERN),DT_PATTERN)),DT_PATTERN)
+ ))) AS c11
+ FROM test.t1
+) AS v2 ON TRUE
+WHERE
+ GET_DAY(DURATION_FROM_INTERVAL(INTERVAL(
+ PARSE_DATETIME(PRINT_DATETIME(CURRENT_DATETIME(),DT_PATTERN),DT_PATTERN),
+ PARSE_DATETIME(v1.c1,DT_PATTERN)
+ ))) <= 30
+ AND
+ (v1.c2 NOT IN ["a","b","c","d"])
+GROUP BY
+ v1.c3, v2.c11;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/fj-dblp-csx-hybrid.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/fj-dblp-csx-hybrid.aql
deleted file mode 100644
index a753907..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/fj-dblp-csx-hybrid.aql
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Multiple fuzzy join on three datasets, with a star join condition.
- * Each star join is composed of a linked join condition to propagate
- * the fuzzy join results.
- * Success : Yes
- */
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPNestedType as closed {
- id: int64,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested: DBLPNestedType
-}
-
-create type CSXNestedType as closed {
- id: int64,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as closed {
- nested: CSXNestedType
-}
-
-create dataset DBLPOpen(DBLPNestedType) primary key id;
-
-create dataset DBLP(DBLPType) primary key nested.id;
-create dataset CSX(CSXType) primary key nested.id;
-
-set import-private-functions 'true';
-set simthreshold "0.5f";
-for $p in dataset DBLP
-for $p1 in dataset CSX
-for $p2 in dataset DBLPOpen
-for $p3 in dataset CSX
-for $p4 in dataset DBLPOpen
-where gram-tokens($p.nested.title, 3, false) ~= gram-tokens($p1.nested.title, 3, false)
-and word-tokens($p1.nested.title) ~= word-tokens($p2.title)
-and word-tokens($p.authors) ~= word-tokens($p3.nested.authors)
-and gram-tokens($p3.nested.misc, 3, false) ~= gram-tokens($p4.misc, 3, false)
-return { "pid": $p.nested.id, "p1id": $p1.nested.id, "p2id": $p2.id, "p3id": $p3.nested.id, "p4id": $p4.id }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/fj-dblp-csx-selflink.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/fj-dblp-csx-selflink.aql
deleted file mode 100644
index 1b3f930..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/fj-dblp-csx-selflink.aql
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy join on two datasets, with a series of link join conditions.
- * The dataset DBLP will fuzzy join with CSX and propagate the results
- * with a series of linked join conditions on CSX.
- * Success : Yes
- */
-drop dataverse test if exists;
-
-create dataverse test;
-
-use dataverse test;
-
-create type DBLPType as open {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as open {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset DBLP(DBLPType) primary key id on group1;
-create dataset CSX(CSXType) primary key id on group1;
-
-write output to asterix_nc1:'rttest/test.adm';
-
-use dataverse test;
-set import-private-functions 'true';
-set simthreshold "0.5f";
-for $p in dataset DBLP
-for $p1 in dataset CSX
-for $p2 in dataset CSX
-for $p3 in dataset CSX
-for $p4 in dataset CSX
-where word-tokens($p.title) ~= word-tokens($p1.title)
-and word-tokens($p2.title) ~= word-tokens($p3.title)
-and gram-tokens($p1.authors, 3, false) ~= gram-tokens($p2.authors, 2, false)
-and gram-tokens($p3.misc, 3, false) ~= gram-tokens($p4.misc, 3, false)
-return { "pid":$p.id, "p1id":$p1.id, "p2id": $p2.id, "p3id": $p3.id, "p4id": $p4.id }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/fj-dblp-csx-simple.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/fj-dblp-csx-simple.aql
deleted file mode 100644
index b0beb4d..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/fj-dblp-csx-simple.aql
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
- /*
- * Description : Two-way fuzzy join on DBLP and CSX with a fuzzy join condition on word tokens of their titles.
- * Success : Yes
- */
-drop dataverse fj-dblp-csx if exists;
-
-create dataverse fj-dblp-csx;
-
-use dataverse fj-dblp-csx;
-
-create type DBLPType as open {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as open {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset DBLP(DBLPType) primary key id on group1;
-create dataset CSX(CSXType) primary key id on group1;
-
-write output to asterix_nc1:'rttest/fj-dblp-csx.adm';
-for $s in dataset('DBLP')
-for $t in dataset('CSX')
-where word-tokens($s.title) ~= word-tokens($t.title)
-return {"sid": $s.id, "tid": $t.id}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/fj-dblp-csx-star.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/fj-dblp-csx-star.aql
deleted file mode 100644
index df0cb86..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/fj-dblp-csx-star.aql
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy join on three datasets, with a set of star join conditions.
- * The CSX and DBLP are used twice and will be propagated onto the output with their ids.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPNestedType as closed {
- id: int64,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested: DBLPNestedType
-}
-
-create type CSXNestedType as closed {
- id: int64,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as closed {
- nested: CSXNestedType
-}
-
-create dataset DBLPOpen(DBLPNestedType) primary key id;
-
-create dataset DBLP(DBLPType) primary key nested.id;
-create dataset CSX(CSXType) primary key nested.id;
-
-set import-private-functions 'true';
-set simthreshold "0.5f";
-for $p in dataset DBLP
-for $p1 in dataset CSX
-for $p2 in dataset DBLPOpen
-for $p3 in dataset CSX
-for $p4 in dataset DBLPOpen
-where word-tokens($p.nested.title) ~= word-tokens($p1.nested.title)
-and word-tokens($p.nested.authors) ~= word-tokens($p3.nested.authors)
-and word-tokens($p.nested.title) ~= word-tokens($p2.title)
-and gram-tokens($p.nested.title, 3, false) ~= gram-tokens($p4.title, 3, false)
-return { "pid": $p.nested.id, "p1id": $p1.nested.id, "p2id": $p2.id, "p3id": $p3.nested.id, "p4id": $p4.id }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/group-by/gby-case-01.3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/group-by/gby-case-01.3.sqlpp
new file mode 100644
index 0000000..f408e69
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/group-by/gby-case-01.3.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create dataset t1(id integer not unknown) open type primary key id;
+
+select x,
+ case when sum(z)=0 then 0 else sum(y*z)/sum(z) end as res
+from t1
+group by x
+order by x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/group-by/gby-case-01.4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/group-by/gby-case-01.4.sqlpp
new file mode 100644
index 0000000..d73995f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/group-by/gby-case-01.4.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create dataset t1(id integer not unknown) open type primary key id;
+
+create function f1() {
+ select x,
+ case when sum(z)=0 then 0 else sum(y*z)/sum(z) end as res
+ from t1
+ group by x
+};
+
+select x, res
+from f1() f
+order by x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_01.aql
deleted file mode 100644
index a9aa579..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_01.aql
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard of their titles' 3-gram tokens.
- * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_01.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $b in dataset('CSX')
-for $a in dataset('DBLP')
-where gram-tokens($a.title, 3, false) ~= gram-tokens($b.title, 3, false) and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_01.aql
deleted file mode 100644
index 07e6c18..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_01.aql
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard-check function of their titles' 3-gram tokens.
- * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_01.adm";
-
-for $b in dataset('CSX')
-for $a in dataset('DBLP')
-where similarity-jaccard-check(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false), 0.5f)[0]
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_01.aql
deleted file mode 100644
index a68d600..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_01.aql
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard function of their titles' 3-gram tokens.
- * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index on DBLP(title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_01.adm";
-
-for $b in dataset('CSX')
-for $a in dataset('DBLP')
-where similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false)) >= 0.5f
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_01.aql
deleted file mode 100644
index 29bd3d0..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-fuzzyeq-jaccard_01.aql
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard of their titles' word tokens.
- * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_01.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $b in dataset('CSX')
-for $a in dataset('DBLP')
-where word-tokens($a.title) ~= word-tokens($b.title) and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_01.aql
deleted file mode 100644
index 989f250..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard-check_01.aql
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard-check function of their titles' word tokens.
- * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_01.adm";
-
-for $b in dataset('CSX')
-for $a in dataset('DBLP')
-where similarity-jaccard-check(word-tokens($a.title), word-tokens($b.title), 0.5f)[0]
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_01.aql
deleted file mode 100644
index e2c373a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/word-jaccard_01.aql
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard function of their titles' word tokens.
- * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index keyword_index on DBLP(title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_01.adm";
-
-for $b in dataset('CSX')
-for $a in dataset('DBLP')
-where similarity-jaccard(word-tokens($a.title), word-tokens($b.title)) >= 0.5f
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/leftouterjoin/loj-03-no-listify.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/leftouterjoin/loj-03-no-listify.sqlpp
new file mode 100644
index 0000000..510e26c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/leftouterjoin/loj-03-no-listify.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Test that listify() is eliminated
+ * on the right side of an outer join
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+CREATE TYPE tasksType AS {
+ id : integer
+};
+
+CREATE DATASET tasks(tasksType) PRIMARY KEY id;
+
+SELECT t0.taskId, t0.cnt_all, t1.cnt_x, t2.cnt_y, t3.cnt_z
+FROM (
+ SELECT taskId, COUNT(1) AS cnt_all FROM tasks GROUP BY taskId ORDER BY taskId
+) AS t0
+LEFT OUTER JOIN (
+ SELECT taskId, COUNT(1) AS cnt_x FROM tasks WHERE status="x" GROUP BY taskId
+) AS t1 ON t0.taskId = t1.taskId
+LEFT OUTER JOIN (
+ SELECT taskId, COUNT(1) AS cnt_y FROM tasks WHERE status="y" GROUP BY taskId
+) AS t2 ON t0.taskId = t2.taskId
+LEFT OUTER JOIN (
+ SELECT taskId, COUNT(1) AS cnt_z FROM tasks WHERE status="z" GROUP BY taskId
+) AS t3 ON t0.taskId = t3.taskId
+ORDER BY t0.taskId;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/leftouterjoin/query-ASTERIXDB-2857.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/leftouterjoin/query-ASTERIXDB-2857.sqlpp
new file mode 100644
index 0000000..2304349
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/leftouterjoin/query-ASTERIXDB-2857.sqlpp
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Test nested loop implementation of left outer join
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type tenkType as closed {
+ unique1 : integer,
+ unique2 : integer,
+ two : integer,
+ four : integer,
+ ten : integer,
+ twenty : integer,
+ hundred : integer,
+ thousand : integer,
+ twothousand : integer,
+ fivethous : integer,
+ tenthous : integer,
+ odd100 : integer,
+ even100 : integer,
+ stringu1 : string,
+ stringu2 : string,
+ string4 : string
+};
+
+create dataset tenk(tenkType) primary key unique2;
+
+SELECT
+ t0.unique1 AS t0_unique1,
+ t1.unique1 AS t1_unique1,
+ t2.unique1 AS t2_unique1
+FROM (
+ SELECT unique1, unique2 FROM tenk WHERE unique2 < 2
+) t0
+INNER JOIN (
+ SELECT unique1, unique2 FROM tenk WHERE unique2 < 4
+) t1 ON t0.unique2 = t1.unique2
+LEFT JOIN (
+ SELECT unique1, unique2 FROM tenk WHERE unique2 < 6
+) t2 ON t0.unique2 + t2.unique2 = 2 * t1.unique2
+ORDER BY t0_unique1, t1_unique1, t2_unique1;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.aql
deleted file mode 100644
index 128b068..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.aql
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
- * Issue : 730, 741
- * Expected Res : Success
- * Date : 8th May 2014
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type TwitterUserType as closed {
- screen-name: string,
- lang: string,
- friends-count: int32,
- statuses-count: int32,
- name: string,
- followers-count: int32
-}
-
-create type TweetMessageNestedType as closed {
- tweetid: int64,
- user: TwitterUserType,
- sender-location: point,
- send-time: datetime,
- referred-topics: {{ string }},
- message-text: string,
- countA: int32,
- countB: int32
-}
-
-create type TweetMessageType as closed {
- nested: TweetMessageNestedType
-}
-
-create dataset TweetMessages(TweetMessageType)
-primary key nested.tweetid;
-
-create index topicKeywordIx on TweetMessages(nested.referred-topics) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-jaccard-check_idx_01.adm";
-
-for $t1 in dataset('TweetMessages')
-where $t1.nested.tweetid > int64("240")
-order by $t1.nested.tweetid
-return {
- "tweet": {"id": $t1.nested.tweetid, "topics" : $t1.nested.referred-topics} ,
- "similar-tweets": for $t2 in dataset('TweetMessages')
- let $sim := similarity-jaccard-check($t1.nested.referred-topics, $t2.nested.referred-topics, 0.5f)
- where $sim[0] and
- $t2.nested.tweetid != $t1.nested.tweetid
- order by $t2.nested.tweetid
- return {"id": $t2.nested.tweetid, "topics" : $t2.nested.referred-topics}
-};
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/ngram-fuzzyeq-jaccard_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/ngram-fuzzyeq-jaccard_01.aql
deleted file mode 100644
index 56e0094..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/ngram-fuzzyeq-jaccard_01.aql
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard of their titles' 3-gram tokens.
- * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPTypetmp as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXTypetmp as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create type CSXType as closed {
- nested : CSXTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create dataset CSX(CSXType) primary key nested.id;
-
-create index ngram_index on CSX(nested.title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_01.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where gram-tokens($a.nested.title, 3, false) ~= gram-tokens($b.nested.title, 3, false) and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/ngram-jaccard-check_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/ngram-jaccard-check_01.aql
deleted file mode 100644
index 482e8c9..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/ngram-jaccard-check_01.aql
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard-check function of their titles' 3-gram tokens.
- * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPTypetmp as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXTypetmp as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create type CSXType as closed {
- nested : CSXTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create dataset CSX(CSXType) primary key nested.id;
-
-create index ngram_index on CSX(nested.title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_01.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where similarity-jaccard-check(gram-tokens($a.nested.title, 3, false), gram-tokens($b.nested.title, 3, false), 0.5f)[0]
- and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/ngram-jaccard-inline.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/ngram-jaccard-inline.aql
deleted file mode 100644
index 7448a06..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/ngram-jaccard-inline.aql
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' 3-gram tokens.
- * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPTypetmp as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create index ngram_index on DBLP(nested.title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-jaccard-inline.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $jacc := similarity-jaccard(gram-tokens($a.nested.title, 3, false), gram-tokens($b.nested.title, 3, false))
-where $jacc >= 0.5f and $a.nested.id < $b.nested.id
-return {"atitle": $a.nested.title, "btitle": $b.nested.title, "jacc": $jacc}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/ngram-jaccard_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/ngram-jaccard_01.aql
deleted file mode 100644
index a0f751d..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/ngram-jaccard_01.aql
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard function of their titles' 3-gram tokens.
- * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPTypetmp as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXTypetmp as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create type CSXType as closed {
- nested : CSXTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create dataset CSX(CSXType) primary key nested.id;
-
-create index ngram_index on CSX(nested.title) type ngram(3);
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_01.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where similarity-jaccard(gram-tokens($a.nested.title, 3, false), gram-tokens($b.nested.title, 3, false)) >= 0.5f
- and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/word-fuzzyeq-jaccard_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/word-fuzzyeq-jaccard_01.aql
deleted file mode 100644
index e45b02d..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/word-fuzzyeq-jaccard_01.aql
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard of their titles' word tokens.
- * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPTypetmp as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXTypetmp as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create type CSXType as closed {
- nested : CSXTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create dataset CSX(CSXType) primary key nested.id;
-
-create index keyword_index on DBLP(nested.title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_01.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $b in dataset('CSX')
-for $a in dataset('DBLP')
-where word-tokens($a.nested.title) ~= word-tokens($b.nested.title) and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/word-jaccard-check-after-btree-access.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/word-jaccard-check-after-btree-access.aql
deleted file mode 100644
index 0fd522d..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/word-jaccard-check-after-btree-access.aql
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy self joins a dataset, TweetMessages, based on the similarity-jaccard-check function of its text-messages' word tokens.
- * TweetMessages has a keyword index on text-message and btree index on the primary key tweetid, and we expect the join to be
- * transformed into btree and inverted indexed nested-loop joins. We test whether the join condition can be transformed into
- * multiple indexed nested loop joins of various type of indexes.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type TwitterUserType as closed {
- screen-name: string,
- lang: string,
- friends-count: int32,
- statuses-count: int32,
- name: string,
- followers-count: int32
-}
-
-create type TweetMessageNestedType as closed {
- tweetid: int64,
- user: TwitterUserType,
- sender-location: point,
- send-time: datetime,
- referred-topics: {{ string }},
- message-text: string,
- countA: int32,
- countB: int32
-}
-
-create type TweetMessageType as closed {
- nested: TweetMessageNestedType
-}
-
-create dataset TweetMessages(TweetMessageType)
-primary key nested.tweetid;
-
-create index twmSndLocIx on TweetMessages(nested.sender-location) type rtree;
-create index msgCountAIx on TweetMessages(nested.countA) type btree;
-create index msgCountBIx on TweetMessages(nested.countB) type btree;
-create index msgTextIx on TweetMessages(nested.message-text) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check-after-btree-access.adm";
-
-for $t1 in dataset('TweetMessages')
-for $t2 in dataset('TweetMessages')
-let $sim := similarity-jaccard-check(word-tokens($t1.nested.message-text), word-tokens($t2.nested.message-text), 0.6f)
-where $sim[0] and $t1.nested.tweetid < int64("20") and $t2.nested.tweetid != $t1.nested.tweetid
-return {
- "t1": $t1.nested.tweetid,
- "t2": $t2.nested.tweetid,
- "sim": $sim[1]
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/word-jaccard-check_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/word-jaccard-check_01.aql
deleted file mode 100644
index 55c2ff4..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/word-jaccard-check_01.aql
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard-check function of their titles' word tokens.
- * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPTypetmp as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXTypetmp as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create type CSXType as closed {
- nested : CSXTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create dataset CSX(CSXType) primary key nested.id;
-
-create index keyword_index on DBLP(nested.title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_01.adm";
-
-for $b in dataset('CSX')
-for $a in dataset('DBLP')
-where similarity-jaccard-check(word-tokens($a.nested.title), word-tokens($b.nested.title), 0.5f)[0]
- and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/word-jaccard-inline.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/word-jaccard-inline.aql
deleted file mode 100644
index 93bcc3e..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/word-jaccard-inline.aql
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' word tokens.
- * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPTypetmp as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create index keyword_index on DBLP(nested.title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_word-jaccard-inline.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $jacc := similarity-jaccard(word-tokens($a.nested.title), word-tokens($b.nested.title))
-where $jacc >= 0.5f and $a.nested.id < $b.nested.id
-return {"atitle": $a.nested.title, "btitle": $b.nested.title, "jacc": $jacc}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/word-jaccard_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/word-jaccard_01.aql
deleted file mode 100644
index acbe3db..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/word-jaccard_01.aql
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard function of their titles' word tokens.
- * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPTypetmp as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXTypetmp as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create type CSXType as closed {
- nested : CSXTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create dataset CSX(CSXType) primary key nested.id;
-
-create index keyword_index on DBLP(nested.title) type keyword;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_01.adm";
-
-for $b in dataset('CSX')
-for $a in dataset('DBLP')
-where similarity-jaccard(word-tokens($a.nested.title), word-tokens($b.nested.title)) >= 0.5f
- and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_01.aql
deleted file mode 100644
index 24dab1c..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_01.aql
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard of their titles' 3-gram tokens.
- * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPTypetmp as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type CSXTypetmp as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create type CSXType as closed {
- nested : CSXTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create dataset CSX(CSXType) primary key nested.id;
-
-create index ngram_index on DBLP(nested.title: string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_01.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $b in dataset('CSX')
-for $a in dataset('DBLP')
-where gram-tokens($a.nested.title, 3, false) ~= gram-tokens($b.nested.title, 3, false) and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_02.aql
deleted file mode 100644
index 2f7c560..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_02.aql
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard of their titles' 3-gram tokens.
- * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPTypetmp as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXTypetmp as open {
- id: int32,
- csxid: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create type CSXType as closed {
- nested : CSXTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create dataset CSX(CSXType) primary key nested.id;
-
-create index ngram_index on CSX(nested.title: string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_01.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $b in dataset('DBLP')
-for $a in dataset('CSX')
-where gram-tokens($a.nested.title, 3, false) ~= gram-tokens($b.nested.title, 3, false) and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_03.aql
deleted file mode 100644
index 5e12569..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_03.aql
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Self joins dataset DBLP, based on ~= using Jaccard of their titles' 3-gram tokens.
- * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPTypetmp as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create index ngram_index on DBLP(nested.title: string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_01.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where gram-tokens($a.nested.title, 3, false) ~= gram-tokens($b.nested.title, 3, false) and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_04.aql
deleted file mode 100644
index cc2f6cc..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-fuzzyeq-jaccard_04.aql
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard of their titles' 3-gram tokens.
- * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPTypetmp as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type CSXTypetmp as open {
- id: int32,
- csxid: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create type CSXType as closed {
- nested : CSXTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create dataset CSX(CSXType) primary key nested.id;
-
-create index ngram_index_DBLP on DBLP(nested.title: string?) type ngram(3) enforced;
-
-create index ngram_index_CSX on CSX(nested.title: string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_01.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where gram-tokens($a.nested.title, 3, false) ~= gram-tokens($b.nested.title, 3, false) and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard-check_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard-check_01.aql
deleted file mode 100644
index da4d914..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard-check_01.aql
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard-check function of their titles' 3-gram tokens.
- * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPTypetmp as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type CSXTypetmp as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create type CSXType as closed {
- nested : CSXTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create dataset CSX(CSXType) primary key nested.id;
-
-create index ngram_index on DBLP(nested.title: string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_01.adm";
-
-for $b in dataset('CSX')
-for $a in dataset('DBLP')
-where similarity-jaccard-check(gram-tokens($a.nested.title, 3, false), gram-tokens($b.nested.title, 3, false), 0.5f)[0]
- and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard-check_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard-check_02.aql
deleted file mode 100644
index 5576768..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard-check_02.aql
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard-check function of their titles' 3-gram tokens.
- * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPTypetmp as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXTypetmp as open {
- id: int32,
- csxid: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create type CSXType as closed {
- nested : CSXTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create dataset CSX(CSXType) primary key nested.id;
-
-create index ngram_index on CSX(nested.title: string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_01.adm";
-
-for $b in dataset('DBLP')
-for $a in dataset('CSX')
-where similarity-jaccard-check(gram-tokens($a.nested.title, 3, false), gram-tokens($b.nested.title, 3, false), 0.5f)[0]
- and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard-check_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard-check_03.aql
deleted file mode 100644
index 32829fb..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard-check_03.aql
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Self joins dataset DBLP, based on the similarity-jaccard-check function of their titles' 3-gram tokens.
- * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPTypetmp as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create index ngram_index on DBLP(nested.title: string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_01.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where similarity-jaccard-check(gram-tokens($a.nested.title, 3, false), gram-tokens($b.nested.title, 3, false), 0.5f)[0]
- and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard-check_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard-check_04.aql
deleted file mode 100644
index dc4c210..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard-check_04.aql
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard-check function of their titles' 3-gram tokens.
- * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPTypetmp as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type CSXTypetmp as open {
- id: int32,
- csxid: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create type CSXType as closed {
- nested : CSXTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create dataset CSX(CSXType) primary key nested.id;
-
-create index ngram_index_DBLP on DBLP(nested.title: string?) type ngram(3) enforced;
-
-create index ngram_index_CSX on CSX(nested.title: string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_01.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where similarity-jaccard-check(gram-tokens($a.nested.title, 3, false), gram-tokens($b.nested.title, 3, false), 0.5f)[0]
- and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard-inline.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard-inline.aql
deleted file mode 100644
index bd599b4..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard-inline.aql
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' 3-gram tokens.
- * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPTypetmp as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create index ngram_index on DBLP(nested.title: string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_ngram-jaccard-inline.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $jacc := similarity-jaccard(gram-tokens($a.nested.title, 3, false), gram-tokens($b.nested.title, 3, false))
-where $jacc >= 0.5f and $a.nested.id < $b.nested.id
-return {"atitle": $a.nested.title, "btitle": $b.nested.title, "jacc": $jacc}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard_01.aql
deleted file mode 100644
index 0106176..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard_01.aql
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard function of their titles' 3-gram tokens.
- * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPTypetmp as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type CSXTypetmp as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create type CSXType as closed {
- nested : CSXTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create dataset CSX(CSXType) primary key nested.id;
-
-create index ngram_index on DBLP(nested.title: string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_01.adm";
-
-for $b in dataset('CSX')
-for $a in dataset('DBLP')
-where similarity-jaccard(gram-tokens($a.nested.title, 3, false), gram-tokens($b.nested.title, 3, false)) >= 0.5f
- and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard_02.aql
deleted file mode 100644
index e9f491b..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard_02.aql
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard function of their titles' 3-gram tokens.
- * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPTypetmp as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXTypetmp as open {
- id: int32,
- csxid: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create type CSXType as closed {
- nested : CSXTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create dataset CSX(CSXType) primary key nested.id;
-
-create index ngram_index on CSX(nested.title: string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_01.adm";
-
-for $b in dataset('DBLP')
-for $a in dataset('CSX')
-where similarity-jaccard(gram-tokens($a.nested.title, 3, false), gram-tokens($b.nested.title, 3, false)) >= 0.5f
- and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard_03.aql
deleted file mode 100644
index e064c03..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard_03.aql
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Self joins dataset DBLP, based on the similarity-jaccard function of their titles' 3-gram tokens.
- * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPTypetmp as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create index ngram_index on DBLP(nested.title: string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_01.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where similarity-jaccard(gram-tokens($a.nested.title, 3, false), gram-tokens($b.nested.title, 3, false)) >= 0.5f
- and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard_04.aql
deleted file mode 100644
index b08eabf..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-jaccard_04.aql
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard function of their titles' 3-gram tokens.
- * DBLP has a 3-gram index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPTypetmp as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type CSXTypetmp as open {
- id: int32,
- csxid: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create type CSXType as closed {
- nested : CSXTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create dataset CSX(CSXType) primary key nested.id;
-
-create index ngram_index_DBLP on DBLP(nested.title: string?) type ngram(3) enforced;
-
-create index ngram_index_CSX on CSX(nested.title: string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_01.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where similarity-jaccard(gram-tokens($a.nested.title, 3, false), gram-tokens($b.nested.title, 3, false)) >= 0.5f
- and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_01.aql
deleted file mode 100644
index 287b338..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_01.aql
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard of their titles' word tokens.
- * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPTypetmp as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type CSXTypetmp as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create type CSXType as closed {
- nested : CSXTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create dataset CSX(CSXType) primary key nested.id;
-
-create index keyword_index on DBLP(nested.title: string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_01.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $b in dataset('CSX')
-for $a in dataset('DBLP')
-where word-tokens($a.nested.title) ~= word-tokens($b.nested.title) and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_02.aql
deleted file mode 100644
index e19b4af..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_02.aql
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard of their titles' word tokens.
- * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPTypetmp as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXTypetmp as open {
- id: int32,
- csxid: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create type CSXType as closed {
- nested : CSXTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create dataset CSX(CSXType) primary key nested.id;
-
-create index keyword_index on CSX(nested.title: string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_01.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $b in dataset('DBLP')
-for $a in dataset('CSX')
-where word-tokens($a.nested.title) ~= word-tokens($b.nested.title) and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_03.aql
deleted file mode 100644
index 5fd64c9..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_03.aql
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Self joins dataset DBLP, based on ~= using Jaccard of their titles' word tokens.
- * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPTypetmp as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create index keyword_index on DBLP(nested.title: string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_01.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where word-tokens($a.nested.title) ~= word-tokens($b.nested.title) and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_04.aql
deleted file mode 100644
index cad00d1..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-fuzzyeq-jaccard_04.aql
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard of their titles' word tokens.
- * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPTypetmp as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type CSXTypetmp as open {
- id: int32,
- csxid: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create type CSXType as closed {
- nested : CSXTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create dataset CSX(CSXType) primary key nested.id;
-
-create index keyword_index_DBLP on DBLP(nested.title: string?) type keyword enforced;
-
-create index keyword_index_CSX on CSX(nested.title: string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_01.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where word-tokens($a.nested.title) ~= word-tokens($b.nested.title) and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard-check-after-btree-access.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard-check-after-btree-access.aql
deleted file mode 100644
index 9ef8bb2..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard-check-after-btree-access.aql
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy self joins a dataset, TweetMessages, based on the similarity-jaccard-check function of its text-messages' word tokens.
- * TweetMessages has a keyword index on text-message and btree index on the primary key tweetid, and we expect the join to be
- * transformed into btree and inverted indexed nested-loop joins. We test whether the join condition can be transformed into
- * multiple indexed nested loop joins of various type of indexes.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type TwitterUserType as closed {
- screen-name: string,
- lang: string,
- friends-count: int32,
- statuses-count: int32,
- name: string,
- followers-count: int32
-}
-
-create type TweetMessageNestedType as open {
- tweetid: int64,
- user: TwitterUserType,
- sender-location: point,
- send-time: datetime,
- referred-topics: {{ string }},
- countA: int32,
- countB: int32
-}
-
-create type TweetMessageType as closed {
- nested: TweetMessageNestedType
-}
-
-create dataset TweetMessages(TweetMessageType)
-primary key nested.tweetid;
-
-create index twmSndLocIx on TweetMessages(nested.sender-location) type rtree;
-create index msgCountAIx on TweetMessages(nested.countA) type btree;
-create index msgCountBIx on TweetMessages(nested.countB) type btree;
-create index msgTextIx on TweetMessages(nested.message-text: string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check-after-btree-access.adm";
-
-for $t1 in dataset('TweetMessages')
-for $t2 in dataset('TweetMessages')
-let $sim := similarity-jaccard-check(word-tokens($t1.nested.message-text), word-tokens($t2.nested.message-text), 0.6f)
-where $sim[0] and $t1.nested.tweetid < int64("20") and $t2.nested.tweetid != $t1.nested.tweetid
-return {
- "t1": $t1.nested.tweetid,
- "t2": $t2.nested.tweetid,
- "sim": $sim[1]
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard-check_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard-check_01.aql
deleted file mode 100644
index 682a754..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard-check_01.aql
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard-check function of their titles' word tokens.
- * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPTypetmp as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type CSXTypetmp as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create type CSXType as closed {
- nested : CSXTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create dataset CSX(CSXType) primary key nested.id;
-
-create index keyword_index on DBLP(nested.title: string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_01.adm";
-
-for $b in dataset('CSX')
-for $a in dataset('DBLP')
-where similarity-jaccard-check(word-tokens($a.nested.title), word-tokens($b.nested.title), 0.5f)[0]
- and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard-check_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard-check_02.aql
deleted file mode 100644
index 583dda6..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard-check_02.aql
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard-check function of their titles' word tokens.
- * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPTypetmp as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXTypetmp as open {
- id: int32,
- csxid: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create type CSXType as closed {
- nested : CSXTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create dataset CSX(CSXType) primary key nested.id;
-
-create index keyword_index on CSX(nested.title: string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_01.adm";
-
-for $b in dataset('DBLP')
-for $a in dataset('CSX')
-where similarity-jaccard-check(word-tokens($a.nested.title), word-tokens($b.nested.title), 0.5f)[0]
- and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard-check_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard-check_03.aql
deleted file mode 100644
index 2cff649..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard-check_03.aql
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Selg joins dataset DBLP, based on the similarity-jaccard-check function of their titles' word tokens.
- * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPTypetmp as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create index keyword_index_DBLP on DBLP(nested.title: string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_01.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where similarity-jaccard-check(word-tokens($a.nested.title), word-tokens($b.nested.title), 0.5f)[0]
- and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard-check_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard-check_04.aql
deleted file mode 100644
index fe7b2f9..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard-check_04.aql
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard-check function of their titles' word tokens.
- * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPTypetmp as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type CSXTypetmp as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create type CSXType as closed {
- nested : CSXTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create dataset CSX(CSXType) primary key nested.id;
-
-create index keyword_index on DBLP(nested.title: string?) type keyword enforced;
-
-create index keyword_index on CSX(nested.title: string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_01.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where similarity-jaccard-check(word-tokens($a.nested.title), word-tokens($b.nested.title), 0.5f)[0]
- and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard-inline.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard-inline.aql
deleted file mode 100644
index 2ec1846..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard-inline.aql
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' word tokens.
- * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * We expect the top-level equi join introduced because of surrogate optimization to be removed, since it is not necessary.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPTypetmp as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create index keyword_index on DBLP(nested.title: string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join-noeqjoin_word-jaccard-inline.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $jacc := similarity-jaccard(word-tokens($a.nested.title), word-tokens($b.nested.title))
-where $jacc >= 0.5f and $a.nested.id < $b.nested.id
-return {"atitle": $a.nested.title, "btitle": $b.nested.title, "jacc": $jacc}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard_01.aql
deleted file mode 100644
index c255829..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard_01.aql
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard function of their titles' word tokens.
- * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPTypetmp as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type CSXTypetmp as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create type CSXType as closed {
- nested : CSXTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create dataset CSX(CSXType) primary key nested.id;
-
-create index keyword_index on DBLP(nested.title: string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_01.adm";
-
-for $b in dataset('CSX')
-for $a in dataset('DBLP')
-where similarity-jaccard(word-tokens($a.nested.title), word-tokens($b.nested.title)) >= 0.5f
- and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard_02.aql
deleted file mode 100644
index 2d52ae9..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard_02.aql
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard function of their titles' word tokens.
- * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPTypetmp as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXTypetmp as open {
- id: int32,
- csxid: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create type CSXType as closed {
- nested : CSXTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create dataset CSX(CSXType) primary key nested.id;
-
-create index keyword_index on CSX(nested.title: string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_01.adm";
-
-for $b in dataset('DBLP')
-for $a in dataset('CSX')
-where similarity-jaccard(word-tokens($a.nested.title), word-tokens($b.nested.title)) >= 0.5f
- and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard_03.aql
deleted file mode 100644
index 456ad8c..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard_03.aql
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Self joins dataset DBLP, based on the similarity-jaccard function of their titles' word tokens.
- * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPTypetmp as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create index keyword_index on DBLP(nested.title: string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_01.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where similarity-jaccard(word-tokens($a.nested.title), word-tokens($b.nested.title)) >= 0.5f
- and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard_04.aql
deleted file mode 100644
index 8812d81..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/word-jaccard_04.aql
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard function of their titles' word tokens.
- * DBLP has a keyword index on title, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPTypetmp as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type CSXTypetmp as open {
- id: int32,
- csxid: string,
- authors: string,
- misc: string
-}
-
-create type DBLPType as closed {
- nested : DBLPTypetmp
-}
-
-create type CSXType as closed {
- nested : CSXTypetmp
-}
-
-create dataset DBLP(DBLPType) primary key nested.id;
-
-create dataset CSX(CSXType) primary key nested.id;
-
-create index keyword_index on DBLP(nested.title: string?) type keyword enforced;
-
-create index keyword_index on CSX(nested.title: string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_01.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where similarity-jaccard(word-tokens($a.nested.title), word-tokens($b.nested.title)) >= 0.5f
- and $a.nested.id < $b.nested.id
-return {"arec": $a, "brec": $b }
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_01.aql
deleted file mode 100644
index 114167e..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_01.aql
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, open DBLP and closed CSX, based on ~= using Jaccard of their titles' 3-gram tokens.
- * DBLP has a 3-gram enforced open index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index on DBLP(title:string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_01.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $b in dataset('CSX')
-for $a in dataset('DBLP')
-where gram-tokens($a.title, 3, false) ~= gram-tokens($b.title, 3, false) and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_02.aql
deleted file mode 100644
index 237ac57..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_02.aql
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, closed DBLP and open CSX, based on ~= using Jaccard their titles' 3-gram tokens.
- * CSX has a 3-gram enforced open index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as open {
- id: int32,
- csxid: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index on CSX(title:string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_02.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where gram-tokens($a.title, 3, false) ~= gram-tokens($b.title, 3, false) and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_03.aql
deleted file mode 100644
index 6ef70de..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_03.aql
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on ~= using Jaccard of its titles' 3-gram tokens.
- * DBLP has a 3-gram enforced open index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title:string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_03.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where gram-tokens($a.title, 3, false) ~= gram-tokens($b.title, 3, false) and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_04.aql
deleted file mode 100644
index 4ddb71f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-fuzzyeq-jaccard_04.aql
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard of their titles' 3-gram tokens.
- * DBLP and CSX both have a 3-gram enforced open index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as open {
- id: int32,
- csxid: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index_DBLP on DBLP(title:string?) type ngram(3) enforced;
-
-create index ngram_index_CSX on CSX(title:string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-fuzzyeq-jaccard_01.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where gram-tokens($a.title, 3, false) ~= gram-tokens($b.title, 3, false) and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard-check_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard-check_01.aql
deleted file mode 100644
index 1277050..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard-check_01.aql
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, open DBLP and closed CSX, based on the similarity-jaccard-check function of their titles' 3-gram tokens.
- * DBLP has a 3-gram enforced open index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index on DBLP(title:string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_01.adm";
-
-for $b in dataset('CSX')
-for $a in dataset('DBLP')
-where similarity-jaccard-check(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false), 0.5f)[0]
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard-check_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard-check_02.aql
deleted file mode 100644
index bbaa284..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard-check_02.aql
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, closed DBLP and open CSX, based the similarity-jaccard-check function of their titles' 3-gram tokens.
- * CSX has a 3-gram enforced open index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as open {
- id: int32,
- csxid: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index on CSX(title:string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_02.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where similarity-jaccard-check(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false), 0.5f)[0]
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard-check_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard-check_03.aql
deleted file mode 100644
index a0f8683..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard-check_03.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy self joins an open dataset DBLP, based on the similarity-jaccard-check function of its titles' 3-gram tokens.
- * DBLP has a 3-gram enforced open index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title:string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where similarity-jaccard-check(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false), 0.5f)[0]
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard-check_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard-check_04.aql
deleted file mode 100644
index 26e2504..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard-check_04.aql
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based the similarity-jaccard-check function of their titles' 3-gram tokens.
- * DBLP and CSX both have a 3-gram enforced open index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as open {
- id: int32,
- csxid: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index_DBLP on DBLP(title:string?) type ngram(3) enforced;
-
-create index ngram_index_CSX on CSX(title:string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_02.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where similarity-jaccard-check(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false), 0.5f)[0]
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard-check_inline_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard-check_inline_03.aql
deleted file mode 100644
index f061fcf..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard-check_inline_03.aql
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' 3-gram tokens.
- * DBLP has a 3-gram enforced open index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title:string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard-check_04.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $jacc := similarity-jaccard-check(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false), 0.5f)
-where $jacc[0] and $a.id < $b.id
-return {"arec": $a, "brec": $b, "jacc": $jacc[1] }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard_01.aql
deleted file mode 100644
index 9f4bd45..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard_01.aql
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard function of their titles' 3-gram tokens.
- * DBLP has a 3-gram enforced open index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index on DBLP(title:string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_01.adm";
-
-for $b in dataset('CSX')
-for $a in dataset('DBLP')
-where similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false)) >= 0.5f
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard_02.aql
deleted file mode 100644
index 82b31c3..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard_02.aql
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based the similarity-jaccard function of their titles' 3-gram tokens.
- * CSX has a 3-gram enforced open index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as open {
- id: int32,
- csxid: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index on CSX(title:string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_02.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false)) >= 0.5f
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard_03.aql
deleted file mode 100644
index e702d56..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard_03.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' 3-gram tokens.
- * DBLP has a 3-gram enforced open index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title:string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false)) >= 0.5f
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard_04.aql
deleted file mode 100644
index e1a9164..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard_04.aql
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based the similarity-jaccard function of their titles' 3-gram tokens.
- * DBLP and CSX both have a 3-gram enforced open index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as open {
- id: int32,
- csxid: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index ngram_index_DBLP on DBLP(title:string?) type ngram(3) enforced;
-
-create index ngram_index_CSX on CSX(title:string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_02.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false)) >= 0.5f
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard_inline_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard_inline_03.aql
deleted file mode 100644
index b634cbf..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-jaccard_inline_03.aql
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' 3-gram tokens.
- * DBLP has a 3-gram enforced open index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-set import-private-functions 'true';
-
-create type DBLPType as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index ngram_index on DBLP(title:string?) type ngram(3) enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_ngram-jaccard_04.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $jacc := similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false))
-where $jacc >= 0.5f and $a.id < $b.id
-return {"arec": $a, "brec": $b, "jacc": $jacc }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_01.aql
deleted file mode 100644
index 3361405..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_01.aql
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard of their titles' word tokens.
- * DBLP has an enforced open keyword index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index keyword_index on DBLP(title:string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_01.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $b in dataset('CSX')
-for $a in dataset('DBLP')
-where word-tokens($a.title) ~= word-tokens($b.title) and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_02.aql
deleted file mode 100644
index 3990354..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_02.aql
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard of their titles' word tokens.
- * CSX has an enforced open keyword index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as open {
- id: int32,
- csxid: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index keyword_index on CSX(title:string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_02.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where word-tokens($a.title) ~= word-tokens($b.title) and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_03.aql
deleted file mode 100644
index 9cccbf0..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_03.aql
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on ~= using Jaccard of its titles' word tokens.
- * DBLP has an enforced open keyword index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title:string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_03.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where word-tokens($a.title) ~= word-tokens($b.title) and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_04.aql
deleted file mode 100644
index 0a0eb66..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-fuzzyeq-jaccard_04.aql
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on ~= using Jaccard of their titles' word tokens.
- * DBLP and CSX both have an enforced open keyword index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as open {
- id: int32,
- csxid: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index keyword_index_DBLP on DBLP(title:string?) type keyword enforced;
-
-create index keyword_index_CSX on CSX(title:string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-fuzzyeq-jaccard_01.adm";
-
-set simfunction 'jaccard';
-set simthreshold '0.5f';
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where word-tokens($a.title) ~= word-tokens($b.title) and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard-check-after-btree-access.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard-check-after-btree-access.aql
deleted file mode 100644
index d090e0b..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard-check-after-btree-access.aql
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy self joins a dataset, TweetMessages, based on the similarity-jaccard-check function of its text-messages' word tokens.
- * TweetMessages has a keyword index on text-message and btree index on the primary key tweetid, and we expect the join to be
- * transformed into btree and inverted indexed nested-loop joins. We test whether the join condition can be transformed into
- * multiple indexed nested loop joins of various type of indexes.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type TwitterUserType as closed {
- screen-name: string,
- lang: string,
- friends-count: int32,
- statuses-count: int32,
- name: string,
- followers-count: int32
-}
-
-create type TweetMessageType as open {
- tweetid: int64,
- user: TwitterUserType,
- sender-location: point,
- send-time: datetime,
- referred-topics: {{ string }},
- countA: int32,
- countB: int32
-}
-
-create dataset TweetMessages(TweetMessageType)
-primary key tweetid;
-
-create index twmSndLocIx on TweetMessages(sender-location) type rtree;
-create index msgCountAIx on TweetMessages(countA) type btree;
-create index msgCountBIx on TweetMessages(countB) type btree;
-create index msgTextIx on TweetMessages(message-text: string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check-after-btree-access.adm";
-
-for $t1 in dataset('TweetMessages')
-for $t2 in dataset('TweetMessages')
-let $sim := similarity-jaccard-check(word-tokens($t1.message-text), word-tokens($t2.message-text), 0.6f)
-where $sim[0] and $t1.tweetid < int64("20") and $t2.tweetid != $t1.tweetid
-return {
- "t1": $t1.tweetid,
- "t2": $t2.tweetid,
- "sim": $sim[1]
-}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard-check_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard-check_01.aql
deleted file mode 100644
index e784002..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard-check_01.aql
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard-check function of their titles' word tokens.
- * DBLP has an enforced open keyword index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index keyword_index on DBLP(title:string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_01.adm";
-
-for $b in dataset('CSX')
-for $a in dataset('DBLP')
-where similarity-jaccard-check(word-tokens($a.title), word-tokens($b.title), 0.5f)[0]
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard-check_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard-check_02.aql
deleted file mode 100644
index 27a170d..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard-check_02.aql
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard-check function of their titles' word tokens.
- * CSX has an enforced open keyword index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as open {
- id: int32,
- csxid: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index keyword_index on CSX(title:string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_02.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where similarity-jaccard-check(word-tokens($a.title), word-tokens($b.title), 0.5f)[0]
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard-check_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard-check_03.aql
deleted file mode 100644
index 7fca777..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard-check_03.aql
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' word tokens.
- * DBLP has an enforced open keyword index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title:string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where similarity-jaccard-check(word-tokens($a.title), word-tokens($b.title), 0.5f)[0]
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard-check_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard-check_04.aql
deleted file mode 100644
index 294732f..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard-check_04.aql
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard-check function of their titles' word tokens.
- * DBLP and CSX both have an enforced open keyword index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as open {
- id: int32,
- csxid: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index keyword_index_DBLP on DBLP(title:string?) type keyword enforced;
-
-create index keyword_index_CSX on CSX(title:string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_01.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where similarity-jaccard-check(word-tokens($a.title), word-tokens($b.title), 0.5f)[0]
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard-check_inline_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard-check_inline_03.aql
deleted file mode 100644
index e2f32f3..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard-check_inline_03.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard-check function of its titles' word tokens.
- * DBLP has an enforced open keyword index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title:string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard-check_04.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $jacc := similarity-jaccard-check(word-tokens($a.title), word-tokens($b.title), 0.5f)
-where $jacc[0] and $a.id < $b.id
-return {"arec": $a, "brec": $b, "jacc": $jacc[1] }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard_01.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard_01.aql
deleted file mode 100644
index 7f2ac1c..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard_01.aql
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard function of their titles' word tokens.
- * DBLP has an enforced open keyword index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as closed {
- id: int32,
- csxid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index keyword_index on DBLP(title:string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_01.adm";
-
-for $b in dataset('CSX')
-for $a in dataset('DBLP')
-where similarity-jaccard(word-tokens($a.title), word-tokens($b.title)) >= 0.5f
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard_02.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard_02.aql
deleted file mode 100644
index a143cff..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard_02.aql
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard function of their titles' word tokens.
- * CSX has an enforced open keyword index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as open {
- id: int32,
- csxid: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index keyword_index on CSX(title:string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_02.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where similarity-jaccard(word-tokens($a.title), word-tokens($b.title)) >= 0.5f
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard_03.aql
deleted file mode 100644
index 744505b..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard_03.aql
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' word tokens.
- * DBLP has an enforced open keyword index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title:string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_03.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-where similarity-jaccard(word-tokens($a.title), word-tokens($b.title)) >= 0.5f
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard_04.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard_04.aql
deleted file mode 100644
index f49113a..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard_04.aql
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy joins two datasets, DBLP and CSX, based on the similarity-jaccard function of their titles' word tokens.
- * DBLP and CSX both have an enforced open keyword index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create type CSXType as open {
- id: int32,
- csxid: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create dataset CSX(CSXType) primary key id;
-
-create index keyword_index_DBLP on DBLP(title:string?) type keyword enforced;
-
-create index keyword_index_CSX on CSX(title:string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_01.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('CSX')
-where similarity-jaccard(word-tokens($a.title), word-tokens($b.title)) >= 0.5f
- and $a.id < $b.id
-return {"arec": $a, "brec": $b }
-
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard_inline_03.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard_inline_03.aql
deleted file mode 100644
index 81f0a77..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/word-jaccard_inline_03.aql
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Fuzzy self joins a dataset, DBLP, based on the similarity-jaccard function of its titles' word tokens.
- * DBLP has an enforced open keyword index on title?, and we expect the join to be transformed into an indexed nested-loop join.
- * We test the inlining of variables that enable the select to be pushed into the join for subsequent optimization with an index.
- * Success : Yes
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type DBLPType as open {
- id: int32,
- dblpid: string,
- authors: string,
- misc: string
-}
-
-create dataset DBLP(DBLPType) primary key id;
-
-create index keyword_index on DBLP(title:string?) type keyword enforced;
-
-write output to asterix_nc1:"rttest/inverted-index-join_word-jaccard_04.adm";
-
-for $a in dataset('DBLP')
-for $b in dataset('DBLP')
-let $jacc := similarity-jaccard(word-tokens($a.title), word-tokens($b.title))
-where $jacc >= 0.5f and $a.id < $b.id
-return {"arec": $a, "brec": $b, "jacc": $jacc }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-04.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-04.sqlpp
new file mode 100644
index 0000000..583dc72
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-04.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Index selection for non-enforced indexes.
+* Expected Res : Success
+*/
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create dataset TestOpen(c_id int64 not unknown) open type primary key c_id;
+
+create index idx_xyz on TestOpen(c_x:string, c_y:int64, c_z:string);
+
+select value t.c_value
+from TestOpen t
+where c_x = "x2" AND c_y > 1 AND c_z = "z2";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-similarity-join-dual-order.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-similarity-join-dual-order.aql
deleted file mode 100644
index ca88a9e..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/similarity/jaccard-similarity-join-dual-order.aql
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Tests the plan of runtimes/fuzzyjoin/basic_1_3_1/6
- * Success : Yes
- */
-
-drop dataverse fuzzyjoin_basic if exists;
-
-create dataverse fuzzyjoin_basic;
-
-use dataverse fuzzyjoin_basic;
-
-create type BasicType as closed {
- id: uuid,
- authors: string
-}
-
-create dataset left(BasicType) primary key id autogenerated;
-create dataset right(BasicType) primary key id autogenerated;
-
-write output to asterix_nc1:"rttest/jaccard-similarity-join-right-ahead.adm";
-
-set import-private-functions 'true'
-
-for $right in dataset('right')
-let $idRight := $right.id
-let $tokensUnrankedRight := word-tokens($right.authors)
-let $lenRight := len($tokensUnrankedRight)
-let $tokensRight :=
- for $tokenUnranked in $tokensUnrankedRight
- for $tokenRanked at $i in
- //
- // -- - Stage 1 - --
- //
- for $r in
- for $orderRight in dataset('right')
- let $rightId := $orderRight.id
- for $orderTokenRight in word-tokens($orderRight.authors)
- /*+ hash */ group by $tokenRightGrouped := $orderTokenRight with $rightId
- return {"rt": $tokenRightGrouped, "rc": count($rightId)}
- for $l in
- for $orderLeft in dataset('left')
- let $leftId := $orderLeft.id
- for $orderTokenLeft in word-tokens($orderLeft.authors)
- /*+ hash */ group by $tokenLeftGrouped := $orderTokenLeft with $leftId
- return {"lt": $tokenLeftGrouped, "lc": count($leftId)}
- where $r.rt = $l.lt
- /*+ inmem 1 302 */ order by $r.rc * $l.lc
- return $r.rt
-
- where $tokenUnranked = /*+ hash-bcast */ $tokenRanked
- order by $i
- return $i
-for $prefixTokenRight in subset-collection($tokensRight, 0, prefix-len-jaccard(len($tokensRight), .8f))
-
-for $left in dataset('left')
-let $idLeft := $left.id
-let $tokensUnrankedLeft := word-tokens($left.authors)
-let $lenLeft := len($tokensUnrankedLeft)
-let $tokensLeft :=
- for $tokenUnranked in $tokensUnrankedLeft
- for $tokenRanked at $i in
- //
- // -- - Stage 1 - --
- //
- for $r in
- for $orderRight in dataset('right')
- let $rightId := $orderRight.id
- for $orderTokenRight in word-tokens($orderRight.authors)
- /*+ hash */ group by $tokenRightGrouped := $orderTokenRight with $rightId
- return {"rt": $tokenRightGrouped, "rc": count($rightId)}
- for $l in
- for $orderLeft in dataset('left')
- let $leftId := $orderLeft.id
- for $orderTokenLeft in word-tokens($orderLeft.authors)
- /*+ hash */ group by $tokenLeftGrouped := $orderTokenLeft with $leftId
- return {"lt": $tokenLeftGrouped, "lc": count($leftId)}
- where $r.rt = $l.lt
- /*+ inmem 1 302 */ order by $r.rc * $l.lc
- return $r.rt
-
- where $tokenUnranked = /*+ hash-bcast */ $tokenRanked
- order by $i
- return $i
-let $actualPrefixLen := prefix-len-jaccard(len($tokensUnrankedLeft), .8f) - len($tokensUnrankedLeft) + len($tokensLeft)
-for $prefixTokenLeft in subset-collection($tokensLeft, 0, $actualPrefixLen)
-
-where $prefixTokenRight = $prefixTokenLeft
-let $sim := similarity-jaccard-prefix($lenRight, $tokensRight, $lenLeft, $tokensLeft, $prefixTokenLeft, .8f)
-where $sim >= .8f
-/*+ hash*/ group by $idRight := $idRight, $idLeft := $idLeft with $sim
-return {'idDBLP': $idRight, 'idCSX': $idLeft, "sim": $sim[0]}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-secondary-btree-index-3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-secondary-btree-index-3.sqlpp
new file mode 100644
index 0000000..d634a26
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-secondary-btree-index-3.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Skip secondary index for IN operator
+ * Expected Res : Success
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type test.TestType as
+{
+ id : integer,
+ fname : string,
+ lname : string
+};
+
+create dataset testdst(TestType) primary key id;
+
+create index sec_Idx on testdst (fname) type btree;
+
+select element emp
+from testdst as emp
+where emp.fname /*+ skip-index */ in ['Max', 'Roger']
+;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-index-01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-index-01.sqlpp
new file mode 100644
index 0000000..bde8e75
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-index-01.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description: Test that BTree index is used in query plan
+ */
+
+// requesttype=application/json
+
+// param $p_fname:json="Julio"
+// param $p_lname:json=null
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type test.Emp as
+ closed {
+ id : bigint,
+ fname : string,
+ lname : string,
+ age : bigint,
+ dept : string
+};
+
+create dataset employee(Emp) primary key id;
+
+create index idx_employee_fname on employee (fname) type btree;
+
+create index idx_employee_lname on employee (lname) type btree;
+
+select l.id, l.fname, l.lname, l.age
+from employee as l
+where
+ ($p_fname is null or l.fname = $p_fname )
+ and
+ ($p_lname is null or l.lname = $p_lname)
+order by l.id;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-index-02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-index-02.sqlpp
new file mode 100644
index 0000000..e74acc5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-index-02.sqlpp
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description: Test that BTree index is used in query plan
+ */
+
+// requesttype=application/json
+
+// param $p_fname:json="Julio"
+// param $p_lname:json="Isa"
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type test.Emp as
+ closed {
+ id : bigint,
+ fname : string,
+ lname : string,
+ age : bigint,
+ dept : string
+};
+
+create dataset employee(Emp) primary key id;
+
+create index idx_employee_fname on employee (fname) type btree;
+
+create index idx_employee_lname on employee (lname) type btree;
+
+select l.id, l.fname, l.lname, l.age
+from employee as l
+where
+ ($p_fname is null or l.fname = $p_fname )
+ and
+ ($p_lname is null or l.lname = $p_lname)
+order by l.id;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-2815-2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-2815-2.sqlpp
new file mode 100644
index 0000000..74034d6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-2815-2.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type t1 as {
+ _id: uuid
+};
+
+create dataset RawTweet(t1) primary key _id autogenerated;
+
+create dataset Evidence(t1) primary key _id autogenerated;
+
+create dataset Verification(t1) primary key _id autogenerated;
+
+select t.id, array_sort(ranks) ranks
+from RawTweet t
+let ranks = (
+ select value rank() over(order by e.url)
+ from Verification v, v.evidence ve, Evidence e
+ where t.id = v.tweet_id and ve = e.ev_id
+)
+order by t.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-2815-3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-2815-3.sqlpp
new file mode 100644
index 0000000..6b0501e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-2815-3.sqlpp
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type t1 as {
+ _id: uuid
+};
+
+create dataset RawTweet(t1) primary key _id autogenerated;
+
+create dataset Evidence(t1) primary key _id autogenerated;
+
+create dataset Verification(t1) primary key _id autogenerated;
+
+select t.id, array_sort(ranks) ranks
+from RawTweet t
+let ranks = (
+ select value rank() over(
+ partition by (tobigint(substring(e.url, -4)) % 2)
+ order by e.url
+ )
+ from Verification v, v.evidence ve, Evidence e
+ where t.id = v.tweet_id and ve = e.ev_id
+)
+order by t.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-2815.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-2815.sqlpp
new file mode 100644
index 0000000..9fd4e02
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-2815.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type t1 as {
+ _id: uuid
+};
+
+create dataset RawTweet(t1) primary key _id autogenerated;
+
+create dataset Evidence(t1) primary key _id autogenerated;
+
+create dataset Verification(t1) primary key _id autogenerated;
+
+select t.id, urls
+from RawTweet t
+let urls = (
+ select distinct value e.url
+ from Verification v, v.evidence ve, Evidence e
+ where t.id = v.tweet_id and ve = e.ev_id
+)
+where array_count(urls) > 2
+order by t.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/unnest_list_in_subplan.aql b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/unnest_list_in_subplan.aql
deleted file mode 100644
index 2c5bf45..0000000
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/unnest_list_in_subplan.aql
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-drop dataverse fuzzyjoin if exists;
-create dataverse fuzzyjoin;
-use dataverse fuzzyjoin;
-
-create type DBLPType as closed {
- id: int32,
- dblpid: string,
- title: string,
- authors: string,
- misc: string
-}
-
-create type TOKENSRANKEDADMType as closed {
- token: int32,
- rank: int32
-}
-
-create nodegroup group1 if not exists on asterix_nc1, asterix_nc2;
-
-create dataset DBLP(DBLPType) primary key id on group1;
-create dataset TOKENSRANKEDADM(TOKENSRANKEDADMType) primary key rank on group1;
-
-write output to asterix_nc1:'rttest/unnest_list_in_subplan.adm';
-
- //
- // -- - Stage 2 - --
- //
- for $paperDBLP in dataset('DBLP')
- let $idDBLP := $paperDBLP.id
- let $tokensUnrankedDBLP := counthashed-word-tokens($paperDBLP.title)
- let $tokensDBLP :=
- for $tokenUnranked in $tokensUnrankedDBLP
- for $tokenRanked in dataset('TOKENSRANKEDADM')
- where $tokenUnranked = $tokenRanked.token
- order by $tokenRanked.rank
- return $tokenRanked.rank
- return {'id': $idDBLP, 'tokens': $tokensDBLP}
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/common-expr-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/common-expr-01.plan
new file mode 100644
index 0000000..83fc968
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/common-expr-01.plan
@@ -0,0 +1,68 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SORT_GROUP_BY[$$176, $$177] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$176, $$177] |PARTITIONED|
+ -- SORT_GROUP_BY[$$159, $$160] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$163] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$163(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$163] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- ASSIGN |UNPARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |UNPARTITIONED|
+ -- RANDOM_MERGE_EXCHANGE |PARTITIONED|
+ -- AGGREGATE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/gby-case-01.3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/gby-case-01.3.plan
new file mode 100644
index 0000000..643e12f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/gby-case-01.3.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$x(ASC) ] |PARTITIONED|
+ -- SORT_GROUP_BY[$$93] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$93] |PARTITIONED|
+ -- SORT_GROUP_BY[$$81] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/gby-case-01.4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/gby-case-01.4.plan
new file mode 100644
index 0000000..27432c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/group-by/gby-case-01.4.plan
@@ -0,0 +1,24 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$x(ASC) ] |PARTITIONED|
+ -- SORT_GROUP_BY[$$120] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$120] |PARTITIONED|
+ -- SORT_GROUP_BY[$$105] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan
index 1a36d3c..8860a1e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan
@@ -78,12 +78,12 @@
-- SORT_MERGE_EXCHANGE [$$130(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$130(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$137] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$136] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$137] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$136] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -133,12 +133,12 @@
-- SORT_MERGE_EXCHANGE [$$130(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$130(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$137] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$136] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$137] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$136] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan
index 1a36d3c..8860a1e 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan
@@ -78,12 +78,12 @@
-- SORT_MERGE_EXCHANGE [$$130(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$130(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$137] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$136] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$137] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$136] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -133,12 +133,12 @@
-- SORT_MERGE_EXCHANGE [$$130(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$130(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$137] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$136] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$137] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$136] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/issue741.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/issue741.plan
index 619b74e..d4ebaa9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/issue741.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/issue741.plan
@@ -84,12 +84,12 @@
-- SORT_MERGE_EXCHANGE [$$140(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$140(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$147] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$146] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$147] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$146] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -143,12 +143,12 @@
-- SORT_MERGE_EXCHANGE [$$140(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$140(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$147] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$146] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$147] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$146] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.plan
index 774794e..56d899c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01.plan
@@ -88,12 +88,12 @@
-- SORT_MERGE_EXCHANGE [$$143(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$143(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$150] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$149] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$150] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$149] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -149,12 +149,12 @@
-- SORT_MERGE_EXCHANGE [$$143(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$143(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$150] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$149] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$150] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$149] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.plan
index 336a4d4..c358e89 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.plan
@@ -94,12 +94,12 @@
-- SORT_MERGE_EXCHANGE [$$143(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$143(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$150] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$149] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$150] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$149] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -155,12 +155,12 @@
-- SORT_MERGE_EXCHANGE [$$143(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$143(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$150] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$149] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$150] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$149] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -269,12 +269,12 @@
-- SORT_MERGE_EXCHANGE [$$143(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$143(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$150] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$149] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$150] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$149] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -330,12 +330,12 @@
-- SORT_MERGE_EXCHANGE [$$143(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$143(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$150] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$149] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$150] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$149] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan
index cbbce46..7cc36bb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan
@@ -66,12 +66,12 @@
-- SORT_MERGE_EXCHANGE [$$117(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$117(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$124] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$123] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$124] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$123] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -121,12 +121,12 @@
-- SORT_MERGE_EXCHANGE [$$117(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$117(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$124] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$123] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$124] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$123] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan
index 79e824e..ddd6bcb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan
@@ -65,12 +65,12 @@
-- SORT_MERGE_EXCHANGE [$$117(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$117(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$124] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$123] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$124] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$123] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -120,12 +120,12 @@
-- SORT_MERGE_EXCHANGE [$$117(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$117(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$124] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$123] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$124] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$123] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan
index 1daa1cf..69c0be7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan
@@ -66,12 +66,12 @@
-- SORT_MERGE_EXCHANGE [$$118(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$118(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$125] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$124] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$125] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$124] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -121,12 +121,12 @@
-- SORT_MERGE_EXCHANGE [$$118(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$118(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$125] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$124] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$125] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$124] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan
index 01e070e..c2938f0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan
@@ -65,12 +65,12 @@
-- SORT_MERGE_EXCHANGE [$$118(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$118(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$125] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$124] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$125] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$124] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -120,12 +120,12 @@
-- SORT_MERGE_EXCHANGE [$$118(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$118(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$125] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$124] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$125] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$124] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan
index 22ad072..0039721 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan
@@ -67,12 +67,12 @@
-- SORT_MERGE_EXCHANGE [$$130(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$130(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$137] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$136] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$137] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$136] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -122,12 +122,12 @@
-- SORT_MERGE_EXCHANGE [$$130(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$130(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$137] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$136] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$137] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$136] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan
index 1daa1cf..69c0be7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan
@@ -66,12 +66,12 @@
-- SORT_MERGE_EXCHANGE [$$118(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$118(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$125] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$124] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$125] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$124] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -121,12 +121,12 @@
-- SORT_MERGE_EXCHANGE [$$118(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$118(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$125] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$124] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$125] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$124] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan
index 01e070e..c2938f0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan
@@ -65,12 +65,12 @@
-- SORT_MERGE_EXCHANGE [$$118(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$118(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$125] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$124] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$125] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$124] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -120,12 +120,12 @@
-- SORT_MERGE_EXCHANGE [$$118(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$118(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$125] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$124] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$125] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$124] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan
index d3acd29..fb60820 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan
@@ -67,12 +67,12 @@
-- SORT_MERGE_EXCHANGE [$$130(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$130(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$137] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$136] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$137] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$136] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -122,12 +122,12 @@
-- SORT_MERGE_EXCHANGE [$$130(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$130(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$137] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$136] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$137] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$136] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan
index cbbce46..7cc36bb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan
@@ -66,12 +66,12 @@
-- SORT_MERGE_EXCHANGE [$$117(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$117(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$124] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$123] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$124] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$123] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -121,12 +121,12 @@
-- SORT_MERGE_EXCHANGE [$$117(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$117(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$124] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$123] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$124] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$123] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan
index 79e824e..ddd6bcb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan
@@ -65,12 +65,12 @@
-- SORT_MERGE_EXCHANGE [$$117(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$117(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$124] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$123] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$124] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$123] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -120,12 +120,12 @@
-- SORT_MERGE_EXCHANGE [$$117(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$117(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$124] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$123] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$124] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$123] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check-after-btree-access.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check-after-btree-access.plan
index 80d465e..70f44c5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check-after-btree-access.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check-after-btree-access.plan
@@ -76,12 +76,12 @@
-- SORT_MERGE_EXCHANGE [$$135(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$135(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$142] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$141] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$142] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$141] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -134,12 +134,12 @@
-- SORT_MERGE_EXCHANGE [$$135(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$135(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$142] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$141] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$142] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$141] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan
index 1daa1cf..69c0be7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan
@@ -66,12 +66,12 @@
-- SORT_MERGE_EXCHANGE [$$118(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$118(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$125] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$124] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$125] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$124] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -121,12 +121,12 @@
-- SORT_MERGE_EXCHANGE [$$118(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$118(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$125] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$124] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$125] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$124] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan
index 01e070e..c2938f0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan
@@ -65,12 +65,12 @@
-- SORT_MERGE_EXCHANGE [$$118(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$118(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$125] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$124] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$125] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$124] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -120,12 +120,12 @@
-- SORT_MERGE_EXCHANGE [$$118(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$118(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$125] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$124] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$125] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$124] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan
index 22ad072..0039721 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan
@@ -67,12 +67,12 @@
-- SORT_MERGE_EXCHANGE [$$130(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$130(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$137] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$136] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$137] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$136] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -122,12 +122,12 @@
-- SORT_MERGE_EXCHANGE [$$130(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$130(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$137] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$136] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$137] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$136] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan
index 1daa1cf..69c0be7 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan
@@ -66,12 +66,12 @@
-- SORT_MERGE_EXCHANGE [$$118(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$118(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$125] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$124] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$125] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$124] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -121,12 +121,12 @@
-- SORT_MERGE_EXCHANGE [$$118(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$118(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$125] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$124] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$125] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$124] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan
index 01e070e..c2938f0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan
@@ -65,12 +65,12 @@
-- SORT_MERGE_EXCHANGE [$$118(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$118(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$125] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$124] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$125] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$124] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -120,12 +120,12 @@
-- SORT_MERGE_EXCHANGE [$$118(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$118(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$125] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$124] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$125] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$124] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan
index d3acd29..fb60820 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan
@@ -67,12 +67,12 @@
-- SORT_MERGE_EXCHANGE [$$130(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$130(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$137] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$136] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$137] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$136] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -122,12 +122,12 @@
-- SORT_MERGE_EXCHANGE [$$130(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$130(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$137] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$136] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$137] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$136] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/leftouterjoin/loj-03-no-listify.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/leftouterjoin/loj-03-no-listify.plan
new file mode 100644
index 0000000..5184b51
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/leftouterjoin/loj-03-no-listify.plan
@@ -0,0 +1,127 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$taskId(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$taskId(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$taskId][$$taskId] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$taskId][$$taskId] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$taskId][$$taskId] |PARTITIONED|
+ -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$taskId(ASC)] HASH:[$$taskId] |PARTITIONED|
+ -- SORT_GROUP_BY[$$263] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$263] |PARTITIONED|
+ -- SORT_GROUP_BY[$$226] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SORT_GROUP_BY[$$265] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$265] |PARTITIONED|
+ -- SORT_GROUP_BY[$$227] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SORT_GROUP_BY[$$267] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$267] |PARTITIONED|
+ -- SORT_GROUP_BY[$$228] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- SORT_GROUP_BY[$$269] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- HASH_PARTITION_EXCHANGE [$$269] |PARTITIONED|
+ -- SORT_GROUP_BY[$$229] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/leftouterjoin/query-ASTERIXDB-2857.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/leftouterjoin/query-ASTERIXDB-2857.plan
new file mode 100644
index 0000000..0df2a49
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/leftouterjoin/query-ASTERIXDB-2857.plan
@@ -0,0 +1,39 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$127(ASC), $$128(ASC), $#3(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$127(ASC), $$128(ASC), $#3(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- NESTED_LOOP |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$118][$$119] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |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/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.plan
index dc19740..8a2df0d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.plan
@@ -98,12 +98,12 @@
-- SORT_MERGE_EXCHANGE [$$170(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$170(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$177] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$176] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$177] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$176] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -158,12 +158,12 @@
-- SORT_MERGE_EXCHANGE [$$170(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$170(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$177] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$176] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$177] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$176] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -275,12 +275,12 @@
-- SORT_MERGE_EXCHANGE [$$170(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$170(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$177] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$176] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$177] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$176] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
@@ -335,12 +335,12 @@
-- SORT_MERGE_EXCHANGE [$$170(ASC), $$tokenGroupped(ASC) ] |PARTITIONED|
-- STABLE_SORT [$$170(ASC), $$tokenGroupped(ASC)] |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EXTERNAL_GROUP_BY[$$177] |PARTITIONED|
+ -- EXTERNAL_GROUP_BY[$$176] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
-- NESTED_TUPLE_SOURCE |LOCAL|
}
- -- HASH_PARTITION_EXCHANGE [$$177] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$176] |PARTITIONED|
-- EXTERNAL_GROUP_BY[$$token] |PARTITIONED|
{
-- AGGREGATE |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-04.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-04.plan
new file mode 100644
index 0000000..f7c87c1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-composite-key-non-enforced/btree-composite-key-non-enforced-04.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 [$$29(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-secondary-btree-index-3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-secondary-btree-index-3.plan
new file mode 100644
index 0000000..87bb65f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-secondary-btree-index-3.plan
@@ -0,0 +1,15 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$14][$$17] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- BROADCAST_EXCHANGE |PARTITIONED|
+ -- UNNEST |UNPARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-index-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-index-01.plan
new file mode 100644
index 0000000..0f02160
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-index-01.plan
@@ -0,0 +1,19 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$28(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$36(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-index-02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-index-02.plan
new file mode 100644
index 0000000..1a1db2c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-index-02.plan
@@ -0,0 +1,30 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$29(ASC) ] |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- INTERSECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$36(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 [$$40(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- BTREE_SEARCH |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2815-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2815-2.plan
new file mode 100644
index 0000000..0a23b38
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2815-2.plan
@@ -0,0 +1,72 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$82(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$82(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$78] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$78][$$89] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- WINDOW_STREAM |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$89(ASC), $$e.url(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$89] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$ve][$$81] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$ve] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$87][$$83] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$87] |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$83] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$81] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2815-3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2815-3.plan
new file mode 100644
index 0000000..3c2caaa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2815-3.plan
@@ -0,0 +1,73 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$88(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$88(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$84] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$84][$$95] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$95] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- WINDOW_STREAM |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$72(ASC), $$95(ASC), $$83(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$72, $$95] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$ve][$$87] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$ve] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$93][$$89] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$93] |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$89] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$87] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2815.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2815.plan
new file mode 100644
index 0000000..ae90721
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2815.plan
@@ -0,0 +1,74 @@
+-- DISTRIBUTE_RESULT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- SORT_MERGE_EXCHANGE [$$75(ASC) ] |PARTITIONED|
+ -- STABLE_SORT [$$75(ASC)] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- STREAM_SELECT |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_CLUSTERED_GROUP_BY[$$70] |PARTITIONED|
+ {
+ -- AGGREGATE |LOCAL|
+ -- STREAM_SELECT |LOCAL|
+ -- NESTED_TUPLE_SOURCE |LOCAL|
+ }
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$70][$$82] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$82] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- PRE_SORTED_DISTINCT_BY |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STABLE_SORT [$$62(ASC), $$82(ASC)] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$62, $$82] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$ve][$$73] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$ve] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- UNNEST |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- HYBRID_HASH_JOIN [$$80][$$76] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$80] |PARTITIONED|
+ -- REPLICATE |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$76] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$73] |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/metrics.xml b/asterixdb/asterix-app/src/test/resources/runtimets/metrics.xml
index da7ba31..a2c3ae5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/metrics.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/metrics.xml
@@ -40,5 +40,10 @@
<output-dir compare="Text">secondary-index-index-only</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="metrics">
+ <compilation-unit name="external-dataset">
+ <output-dir compare="Text">external-dataset</output-dir>
+ </compilation-unit>
+ </test-case>
</test-group>
</test-suite>
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/only_external_dataset.xml b/asterixdb/asterix-app/src/test/resources/runtimets/only_external_dataset.xml
new file mode 100644
index 0000000..334dd52
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/only_external_dataset.xml
@@ -0,0 +1,23 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements. See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership. The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License. You may obtain a copy of the License at
+ !
+ ! http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied. See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp">
+ <test-group name="failed">
+ </test-group>
+</test-suite>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/avg_mixed/avg_mixed.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/avg_mixed/avg_mixed.1.ddl.sqlpp
deleted file mode 100644
index 5dc84e2..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/avg_mixed/avg_mixed.1.ddl.sqlpp
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
-* Description : Run array_avg over an ordered list with mixed types
-* Expected Res : Failure
-* Date : Feb 7th 2014
-*/
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/avg_mixed/avg_mixed.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/avg_mixed/avg_mixed.2.update.sqlpp
deleted file mode 100644
index bd244d0..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/avg_mixed/avg_mixed.2.update.sqlpp
+++ /dev/null
@@ -1,19 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/avg_mixed/avg_mixed.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/avg_mixed/avg_mixed.3.query.sqlpp
index ff0badd..03b82fe 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/avg_mixed/avg_mixed.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/avg_mixed/avg_mixed.3.query.sqlpp
@@ -21,8 +21,9 @@
* Expected Res : Failure
* Date : Feb 7th 2014
*/
+-- param max-warnings:json=1000
select element array_avg((
select element x
- from [float('2.0'),'hello world',93847382783847382,date('2013-01-01')] as x
+ from [float('2.0'),'hello world',10,date('2013-01-01')] as x
));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_double/serial_avg_double.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_double/serial_avg_double.1.ddl.sqlpp
new file mode 100644
index 0000000..da0bd7b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_double/serial_avg_double.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : double
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_double/serial_avg_double.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_double/serial_avg_double.2.update.sqlpp
new file mode 100644
index 0000000..88d820c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_double/serial_avg_double.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':double(4.32),'valplus':double(473847.0)};
+insert into Test
+select element {'id':1,'gid':1,'val':double(5.32)};
+insert into Test
+select element {'id':2,'gid':1,'val':double(6.32),'valplus':double(38473827484738239.0)};
+insert into Test
+select element {'id':3,'gid':1,'val':double(4.32),'valplus':double(678900.0)};
+insert into Test
+select element {'id':4,'gid':1,'val':double(4.32),'valplus':null};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_double/serial_avg_double.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_double/serial_avg_double.3.query.sqlpp
new file mode 100644
index 0000000..c141c2b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_double/serial_avg_double.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, avg(t.val) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_double_null/serial_avg_double_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_double_null/serial_avg_double_null.1.ddl.sqlpp
new file mode 100644
index 0000000..da0bd7b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_double_null/serial_avg_double_null.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : double
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_double_null/serial_avg_double_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_double_null/serial_avg_double_null.2.update.sqlpp
new file mode 100644
index 0000000..1cff84e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_double_null/serial_avg_double_null.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':double(4.32),'valplus':double(100.0)};
+insert into Test
+select element {'id':1,'gid':1,'val':double(5.32)};
+insert into Test
+select element {'id':2,'gid':1,'val':double(6.32),'valplus':"str"};
+insert into Test
+select element {'id':3,'gid':1,'val':double(4.32),'valplus':double(200.0)};
+insert into Test
+select element {'id':4,'gid':1,'val':double(4.32),'valplus':null};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_double_null/serial_avg_double_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_double_null/serial_avg_double_null.3.query.sqlpp
new file mode 100644
index 0000000..acbf607
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_double_null/serial_avg_double_null.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, avg(t.valplus) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_empty/serial_avg_empty.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_empty/serial_avg_empty.1.ddl.sqlpp
new file mode 100644
index 0000000..da0bd7b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_empty/serial_avg_empty.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : double
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_empty/serial_avg_empty.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_empty/serial_avg_empty.2.update.sqlpp
new file mode 100644
index 0000000..042f3ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_empty/serial_avg_empty.2.update.sqlpp
@@ -0,0 +1,18 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_empty/serial_avg_empty.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_empty/serial_avg_empty.3.query.sqlpp
new file mode 100644
index 0000000..c141c2b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_empty/serial_avg_empty.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, avg(t.val) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_float/serial_avg_float.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_float/serial_avg_float.1.ddl.sqlpp
new file mode 100644
index 0000000..ddac6a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_float/serial_avg_float.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : float
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_float/serial_avg_float.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_float/serial_avg_float.2.update.sqlpp
new file mode 100644
index 0000000..40641b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_float/serial_avg_float.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':float('4.32'),'valplus':float('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':float('5.32')};
+insert into Test
+select element {'id':2,'gid':1,'val':float('6.32'),'valplus':float('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':float('4.32'),'valplus':float('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':float('4.32'),'valplus':null};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_float/serial_avg_float.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_float/serial_avg_float.3.query.sqlpp
new file mode 100644
index 0000000..c141c2b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_float/serial_avg_float.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, avg(t.val) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_float_null/serial_avg_float_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_float_null/serial_avg_float_null.1.ddl.sqlpp
new file mode 100644
index 0000000..ddac6a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_float_null/serial_avg_float_null.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : float
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_float_null/serial_avg_float_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_float_null/serial_avg_float_null.2.update.sqlpp
new file mode 100644
index 0000000..40641b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_float_null/serial_avg_float_null.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':float('4.32'),'valplus':float('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':float('5.32')};
+insert into Test
+select element {'id':2,'gid':1,'val':float('6.32'),'valplus':float('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':float('4.32'),'valplus':float('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':float('4.32'),'valplus':null};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_float_null/serial_avg_float_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_float_null/serial_avg_float_null.3.query.sqlpp
new file mode 100644
index 0000000..acbf607
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_float_null/serial_avg_float_null.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, avg(t.valplus) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int16/serial_avg_int16.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int16/serial_avg_int16.1.ddl.sqlpp
new file mode 100644
index 0000000..0c0bfee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int16/serial_avg_int16.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : smallint
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int16/serial_avg_int16.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int16/serial_avg_int16.2.update.sqlpp
new file mode 100644
index 0000000..6210af4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int16/serial_avg_int16.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':smallint('4'),'valplus':smallint('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':smallint('5')};
+insert into Test
+select element {'id':2,'gid':1,'val':smallint('6'),'valplus':smallint('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':smallint('4'),'valplus':smallint('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':smallint('4'),'valplus':null};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int16/serial_avg_int16.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int16/serial_avg_int16.3.query.sqlpp
new file mode 100644
index 0000000..c141c2b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int16/serial_avg_int16.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, avg(t.val) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int16_null/serial_avg_int16_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int16_null/serial_avg_int16_null.1.ddl.sqlpp
new file mode 100644
index 0000000..0c0bfee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int16_null/serial_avg_int16_null.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : smallint
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int16_null/serial_avg_int16_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int16_null/serial_avg_int16_null.2.update.sqlpp
new file mode 100644
index 0000000..6210af4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int16_null/serial_avg_int16_null.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':smallint('4'),'valplus':smallint('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':smallint('5')};
+insert into Test
+select element {'id':2,'gid':1,'val':smallint('6'),'valplus':smallint('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':smallint('4'),'valplus':smallint('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':smallint('4'),'valplus':null};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int16_null/serial_avg_int16_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int16_null/serial_avg_int16_null.3.query.sqlpp
new file mode 100644
index 0000000..acbf607
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int16_null/serial_avg_int16_null.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, avg(t.valplus) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int32/serial_avg_int32.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int32/serial_avg_int32.1.ddl.sqlpp
new file mode 100644
index 0000000..07312bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int32/serial_avg_int32.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : integer
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int32/serial_avg_int32.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int32/serial_avg_int32.2.update.sqlpp
new file mode 100644
index 0000000..a248a0f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int32/serial_avg_int32.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':integer('4'),'valplus':integer('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':integer('5')};
+insert into Test
+select element {'id':2,'gid':1,'val':integer('6'),'valplus':integer('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':integer('4'),'valplus':integer('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':integer('4'),'valplus':null};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int32/serial_avg_int32.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int32/serial_avg_int32.3.query.sqlpp
new file mode 100644
index 0000000..c141c2b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int32/serial_avg_int32.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, avg(t.val) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int32_null/serial_avg_int32_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int32_null/serial_avg_int32_null.1.ddl.sqlpp
new file mode 100644
index 0000000..07312bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int32_null/serial_avg_int32_null.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : integer
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int32_null/serial_avg_int32_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int32_null/serial_avg_int32_null.2.update.sqlpp
new file mode 100644
index 0000000..a248a0f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int32_null/serial_avg_int32_null.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':integer('4'),'valplus':integer('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':integer('5')};
+insert into Test
+select element {'id':2,'gid':1,'val':integer('6'),'valplus':integer('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':integer('4'),'valplus':integer('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':integer('4'),'valplus':null};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int32_null/serial_avg_int32_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int32_null/serial_avg_int32_null.3.query.sqlpp
new file mode 100644
index 0000000..acbf607
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int32_null/serial_avg_int32_null.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, avg(t.valplus) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int64/serial_avg_int64.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int64/serial_avg_int64.1.ddl.sqlpp
new file mode 100644
index 0000000..5e9972c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int64/serial_avg_int64.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : bigint
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int64/serial_avg_int64.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int64/serial_avg_int64.2.update.sqlpp
new file mode 100644
index 0000000..b460af2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int64/serial_avg_int64.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':bigint('4'),'valplus':bigint('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':bigint('5')};
+insert into Test
+select element {'id':2,'gid':1,'val':bigint('6'),'valplus':bigint('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':bigint('4'),'valplus':bigint('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':bigint('4'),'valplus':null};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int64/serial_avg_int64.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int64/serial_avg_int64.3.query.sqlpp
new file mode 100644
index 0000000..c141c2b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int64/serial_avg_int64.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, avg(t.val) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int64_null/serial_avg_int64_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int64_null/serial_avg_int64_null.1.ddl.sqlpp
new file mode 100644
index 0000000..5e9972c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int64_null/serial_avg_int64_null.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : bigint
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int64_null/serial_avg_int64_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int64_null/serial_avg_int64_null.2.update.sqlpp
new file mode 100644
index 0000000..b460af2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int64_null/serial_avg_int64_null.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':bigint('4'),'valplus':bigint('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':bigint('5')};
+insert into Test
+select element {'id':2,'gid':1,'val':bigint('6'),'valplus':bigint('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':bigint('4'),'valplus':bigint('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':bigint('4'),'valplus':null};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int64_null/serial_avg_int64_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int64_null/serial_avg_int64_null.3.query.sqlpp
new file mode 100644
index 0000000..acbf607
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int64_null/serial_avg_int64_null.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, avg(t.valplus) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int8/serial_avg_int8.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int8/serial_avg_int8.1.ddl.sqlpp
new file mode 100644
index 0000000..c45dbfe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int8/serial_avg_int8.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : tinyint
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int8/serial_avg_int8.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int8/serial_avg_int8.2.update.sqlpp
new file mode 100644
index 0000000..eb41173
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int8/serial_avg_int8.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':tinyint('100'),'valplus':tinyint('100')};
+insert into Test
+select element {'id':1,'gid':1,'val':tinyint('100')};
+insert into Test
+select element {'id':2,'gid':1,'val':tinyint('90'),'valplus':tinyint('100')};
+insert into Test
+select element {'id':3,'gid':1,'val':tinyint('40'),'valplus':tinyint('100')};
+insert into Test
+select element {'id':4,'gid':1,'val':tinyint('40'),'valplus':null};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int8/serial_avg_int8.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int8/serial_avg_int8.3.query.sqlpp
new file mode 100644
index 0000000..c141c2b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int8/serial_avg_int8.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, avg(t.val) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int8_null/serial_avg_int8_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int8_null/serial_avg_int8_null.1.ddl.sqlpp
new file mode 100644
index 0000000..c45dbfe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int8_null/serial_avg_int8_null.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : tinyint
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int8_null/serial_avg_int8_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int8_null/serial_avg_int8_null.2.update.sqlpp
new file mode 100644
index 0000000..eb41173
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int8_null/serial_avg_int8_null.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':tinyint('100'),'valplus':tinyint('100')};
+insert into Test
+select element {'id':1,'gid':1,'val':tinyint('100')};
+insert into Test
+select element {'id':2,'gid':1,'val':tinyint('90'),'valplus':tinyint('100')};
+insert into Test
+select element {'id':3,'gid':1,'val':tinyint('40'),'valplus':tinyint('100')};
+insert into Test
+select element {'id':4,'gid':1,'val':tinyint('40'),'valplus':null};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int8_null/serial_avg_int8_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int8_null/serial_avg_int8_null.3.query.sqlpp
new file mode 100644
index 0000000..acbf607
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_int8_null/serial_avg_int8_null.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, avg(t.valplus) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_mixed/serial_avg_mixed.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_mixed/serial_avg_mixed.1.ddl.sqlpp
new file mode 100644
index 0000000..da0bd7b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_mixed/serial_avg_mixed.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : double
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_mixed/serial_avg_mixed.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_mixed/serial_avg_mixed.2.update.sqlpp
new file mode 100644
index 0000000..8c0b731
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_mixed/serial_avg_mixed.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':double(4.32),'valplus':float('2.0')};
+insert into Test
+select element {'id':1,'gid':1,'val':double(5.32)};
+insert into Test
+select element {'id':2,'gid':1,'val':double(6.32),'valplus':'hello world'};
+insert into Test
+select element {'id':3,'gid':1,'val':double(4.32),'valplus':double('3.0')};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_mixed/serial_avg_mixed.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_mixed/serial_avg_mixed.3.query.sqlpp
new file mode 100644
index 0000000..dcb6501
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-sql/serial_avg_mixed/serial_avg_mixed.3.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ /*
+ * Description : Run serial-sum over an ordered list with mixed types
+ * Expected Res : Failure
+ * Date : March 5th 2018
+ */
+
+-- param max-warnings:json=1000
+
+use test;
+
+select gid, array_avg((select value g.valplus from g)) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid
+group as g(valplus as valplus);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_mixed/avg_mixed.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_mixed/avg_mixed.1.ddl.sqlpp
deleted file mode 100644
index 095f14e..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_mixed/avg_mixed.1.ddl.sqlpp
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
-* Description : Run avg over an ordered list with mixed types
-* Expected Res : Failure
-* Date : Jun 2nd 2013
-*/
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_mixed/avg_mixed.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_mixed/avg_mixed.2.update.sqlpp
deleted file mode 100644
index bd244d0..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_mixed/avg_mixed.2.update.sqlpp
+++ /dev/null
@@ -1,19 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_mixed/avg_mixed.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_mixed/avg_mixed.3.query.sqlpp
index d07bdc8..5e6596c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_mixed/avg_mixed.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/avg_mixed/avg_mixed.3.query.sqlpp
@@ -21,6 +21,7 @@
* Expected Res : Failure
* Date : Jun 2nd 2013
*/
+-- param max-warnings:json=1000
select element strict_avg((
select element x
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_double/serial_avg_double.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_double/serial_avg_double.1.ddl.sqlpp
new file mode 100644
index 0000000..da0bd7b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_double/serial_avg_double.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : double
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_double/serial_avg_double.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_double/serial_avg_double.2.update.sqlpp
new file mode 100644
index 0000000..88d820c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_double/serial_avg_double.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':double(4.32),'valplus':double(473847.0)};
+insert into Test
+select element {'id':1,'gid':1,'val':double(5.32)};
+insert into Test
+select element {'id':2,'gid':1,'val':double(6.32),'valplus':double(38473827484738239.0)};
+insert into Test
+select element {'id':3,'gid':1,'val':double(4.32),'valplus':double(678900.0)};
+insert into Test
+select element {'id':4,'gid':1,'val':double(4.32),'valplus':null};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_double/serial_avg_double.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_double/serial_avg_double.3.query.sqlpp
new file mode 100644
index 0000000..09c3106
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_double/serial_avg_double.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_avg((select value g.val from g)) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(val as val);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_double_null/serial_avg_double_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_double_null/serial_avg_double_null.1.ddl.sqlpp
new file mode 100644
index 0000000..da0bd7b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_double_null/serial_avg_double_null.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : double
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_double_null/serial_avg_double_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_double_null/serial_avg_double_null.2.update.sqlpp
new file mode 100644
index 0000000..88d820c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_double_null/serial_avg_double_null.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':double(4.32),'valplus':double(473847.0)};
+insert into Test
+select element {'id':1,'gid':1,'val':double(5.32)};
+insert into Test
+select element {'id':2,'gid':1,'val':double(6.32),'valplus':double(38473827484738239.0)};
+insert into Test
+select element {'id':3,'gid':1,'val':double(4.32),'valplus':double(678900.0)};
+insert into Test
+select element {'id':4,'gid':1,'val':double(4.32),'valplus':null};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_double_null/serial_avg_double_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_double_null/serial_avg_double_null.3.query.sqlpp
new file mode 100644
index 0000000..35ff90f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_double_null/serial_avg_double_null.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_avg((select value g.valplus from g)) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(valplus as valplus);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_empty/serial_avg_empty.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_empty/serial_avg_empty.1.ddl.sqlpp
new file mode 100644
index 0000000..da0bd7b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_empty/serial_avg_empty.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : double
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_empty/serial_avg_empty.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_empty/serial_avg_empty.2.update.sqlpp
new file mode 100644
index 0000000..042f3ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_empty/serial_avg_empty.2.update.sqlpp
@@ -0,0 +1,18 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_empty/serial_avg_empty.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_empty/serial_avg_empty.3.query.sqlpp
new file mode 100644
index 0000000..09c3106
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_empty/serial_avg_empty.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_avg((select value g.val from g)) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(val as val);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_float/serial_avg_float.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_float/serial_avg_float.1.ddl.sqlpp
new file mode 100644
index 0000000..ddac6a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_float/serial_avg_float.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : float
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_float/serial_avg_float.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_float/serial_avg_float.2.update.sqlpp
new file mode 100644
index 0000000..40641b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_float/serial_avg_float.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':float('4.32'),'valplus':float('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':float('5.32')};
+insert into Test
+select element {'id':2,'gid':1,'val':float('6.32'),'valplus':float('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':float('4.32'),'valplus':float('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':float('4.32'),'valplus':null};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_float/serial_avg_float.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_float/serial_avg_float.3.query.sqlpp
new file mode 100644
index 0000000..09c3106
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_float/serial_avg_float.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_avg((select value g.val from g)) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(val as val);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_float_null/serial_avg_float_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_float_null/serial_avg_float_null.1.ddl.sqlpp
new file mode 100644
index 0000000..ddac6a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_float_null/serial_avg_float_null.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : float
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_float_null/serial_avg_float_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_float_null/serial_avg_float_null.2.update.sqlpp
new file mode 100644
index 0000000..40641b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_float_null/serial_avg_float_null.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':float('4.32'),'valplus':float('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':float('5.32')};
+insert into Test
+select element {'id':2,'gid':1,'val':float('6.32'),'valplus':float('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':float('4.32'),'valplus':float('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':float('4.32'),'valplus':null};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_float_null/serial_avg_float_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_float_null/serial_avg_float_null.3.query.sqlpp
new file mode 100644
index 0000000..35ff90f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_float_null/serial_avg_float_null.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_avg((select value g.valplus from g)) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(valplus as valplus);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int16/serial_avg_int16.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int16/serial_avg_int16.1.ddl.sqlpp
new file mode 100644
index 0000000..0c0bfee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int16/serial_avg_int16.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : smallint
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int16/serial_avg_int16.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int16/serial_avg_int16.2.update.sqlpp
new file mode 100644
index 0000000..6210af4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int16/serial_avg_int16.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':smallint('4'),'valplus':smallint('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':smallint('5')};
+insert into Test
+select element {'id':2,'gid':1,'val':smallint('6'),'valplus':smallint('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':smallint('4'),'valplus':smallint('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':smallint('4'),'valplus':null};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int16/serial_avg_int16.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int16/serial_avg_int16.3.query.sqlpp
new file mode 100644
index 0000000..09c3106
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int16/serial_avg_int16.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_avg((select value g.val from g)) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(val as val);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int16_null/serial_avg_int16_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int16_null/serial_avg_int16_null.1.ddl.sqlpp
new file mode 100644
index 0000000..0c0bfee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int16_null/serial_avg_int16_null.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : smallint
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int16_null/serial_avg_int16_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int16_null/serial_avg_int16_null.2.update.sqlpp
new file mode 100644
index 0000000..6210af4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int16_null/serial_avg_int16_null.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':smallint('4'),'valplus':smallint('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':smallint('5')};
+insert into Test
+select element {'id':2,'gid':1,'val':smallint('6'),'valplus':smallint('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':smallint('4'),'valplus':smallint('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':smallint('4'),'valplus':null};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int16_null/serial_avg_int16_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int16_null/serial_avg_int16_null.3.query.sqlpp
new file mode 100644
index 0000000..35ff90f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int16_null/serial_avg_int16_null.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_avg((select value g.valplus from g)) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(valplus as valplus);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int32/serial_avg_int32.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int32/serial_avg_int32.1.ddl.sqlpp
new file mode 100644
index 0000000..07312bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int32/serial_avg_int32.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : integer
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int32/serial_avg_int32.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int32/serial_avg_int32.2.update.sqlpp
new file mode 100644
index 0000000..a248a0f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int32/serial_avg_int32.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':integer('4'),'valplus':integer('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':integer('5')};
+insert into Test
+select element {'id':2,'gid':1,'val':integer('6'),'valplus':integer('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':integer('4'),'valplus':integer('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':integer('4'),'valplus':null};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int32/serial_avg_int32.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int32/serial_avg_int32.3.query.sqlpp
new file mode 100644
index 0000000..09c3106
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int32/serial_avg_int32.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_avg((select value g.val from g)) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(val as val);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int32_null/serial_avg_int32_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int32_null/serial_avg_int32_null.1.ddl.sqlpp
new file mode 100644
index 0000000..07312bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int32_null/serial_avg_int32_null.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : integer
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int32_null/serial_avg_int32_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int32_null/serial_avg_int32_null.2.update.sqlpp
new file mode 100644
index 0000000..a248a0f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int32_null/serial_avg_int32_null.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':integer('4'),'valplus':integer('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':integer('5')};
+insert into Test
+select element {'id':2,'gid':1,'val':integer('6'),'valplus':integer('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':integer('4'),'valplus':integer('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':integer('4'),'valplus':null};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int32_null/serial_avg_int32_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int32_null/serial_avg_int32_null.3.query.sqlpp
new file mode 100644
index 0000000..35ff90f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int32_null/serial_avg_int32_null.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_avg((select value g.valplus from g)) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(valplus as valplus);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int64/serial_avg_int64.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int64/serial_avg_int64.1.ddl.sqlpp
new file mode 100644
index 0000000..5e9972c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int64/serial_avg_int64.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : bigint
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int64/serial_avg_int64.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int64/serial_avg_int64.2.update.sqlpp
new file mode 100644
index 0000000..b460af2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int64/serial_avg_int64.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':bigint('4'),'valplus':bigint('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':bigint('5')};
+insert into Test
+select element {'id':2,'gid':1,'val':bigint('6'),'valplus':bigint('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':bigint('4'),'valplus':bigint('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':bigint('4'),'valplus':null};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int64/serial_avg_int64.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int64/serial_avg_int64.3.query.sqlpp
new file mode 100644
index 0000000..09c3106
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int64/serial_avg_int64.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_avg((select value g.val from g)) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(val as val);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int64_null/serial_avg_int64_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int64_null/serial_avg_int64_null.1.ddl.sqlpp
new file mode 100644
index 0000000..5e9972c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int64_null/serial_avg_int64_null.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : bigint
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int64_null/serial_avg_int64_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int64_null/serial_avg_int64_null.2.update.sqlpp
new file mode 100644
index 0000000..b460af2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int64_null/serial_avg_int64_null.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':bigint('4'),'valplus':bigint('1')};
+insert into Test
+select element {'id':1,'gid':1,'val':bigint('5')};
+insert into Test
+select element {'id':2,'gid':1,'val':bigint('6'),'valplus':bigint('2')};
+insert into Test
+select element {'id':3,'gid':1,'val':bigint('4'),'valplus':bigint('3')};
+insert into Test
+select element {'id':4,'gid':1,'val':bigint('4'),'valplus':null};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int64_null/serial_avg_int64_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int64_null/serial_avg_int64_null.3.query.sqlpp
new file mode 100644
index 0000000..35ff90f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int64_null/serial_avg_int64_null.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_avg((select value g.valplus from g)) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(valplus as valplus);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int8/serial_avg_int8.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int8/serial_avg_int8.1.ddl.sqlpp
new file mode 100644
index 0000000..c45dbfe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int8/serial_avg_int8.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : tinyint
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int8/serial_avg_int8.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int8/serial_avg_int8.2.update.sqlpp
new file mode 100644
index 0000000..eb41173
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int8/serial_avg_int8.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':tinyint('100'),'valplus':tinyint('100')};
+insert into Test
+select element {'id':1,'gid':1,'val':tinyint('100')};
+insert into Test
+select element {'id':2,'gid':1,'val':tinyint('90'),'valplus':tinyint('100')};
+insert into Test
+select element {'id':3,'gid':1,'val':tinyint('40'),'valplus':tinyint('100')};
+insert into Test
+select element {'id':4,'gid':1,'val':tinyint('40'),'valplus':null};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int8/serial_avg_int8.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int8/serial_avg_int8.3.query.sqlpp
new file mode 100644
index 0000000..09c3106
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int8/serial_avg_int8.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_avg((select value g.val from g)) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(val as val);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int8_null/serial_avg_int8_null.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int8_null/serial_avg_int8_null.1.ddl.sqlpp
new file mode 100644
index 0000000..c45dbfe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int8_null/serial_avg_int8_null.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : tinyint
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int8_null/serial_avg_int8_null.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int8_null/serial_avg_int8_null.2.update.sqlpp
new file mode 100644
index 0000000..eb41173
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int8_null/serial_avg_int8_null.2.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':tinyint('100'),'valplus':tinyint('100')};
+insert into Test
+select element {'id':1,'gid':1,'val':tinyint('100')};
+insert into Test
+select element {'id':2,'gid':1,'val':tinyint('90'),'valplus':tinyint('100')};
+insert into Test
+select element {'id':3,'gid':1,'val':tinyint('40'),'valplus':tinyint('100')};
+insert into Test
+select element {'id':4,'gid':1,'val':tinyint('40'),'valplus':null};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int8_null/serial_avg_int8_null.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int8_null/serial_avg_int8_null.3.query.sqlpp
new file mode 100644
index 0000000..35ff90f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_int8_null/serial_avg_int8_null.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select gid, strict_avg((select value g.valplus from g)) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(valplus as valplus);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_mixed/serial_avg_mixed.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_mixed/serial_avg_mixed.1.ddl.sqlpp
new file mode 100644
index 0000000..da0bd7b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_mixed/serial_avg_mixed.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+
+create type test.TestType as
+{
+ id : bigint,
+ gid : bigint,
+ val : double
+};
+
+create dataset Test(TestType) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_mixed/serial_avg_mixed.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_mixed/serial_avg_mixed.2.update.sqlpp
new file mode 100644
index 0000000..02788ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_mixed/serial_avg_mixed.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into Test
+select element {'id':0,'gid':1,'val':double(4.32),'valplus':float('2.0')};
+insert into Test
+select element {'id':1,'gid':1,'val':double(5.32), 'valplus':int32("15")};
+insert into Test
+select element {'id':2,'gid':1,'val':double(6.32),'valplus':'hello world'};
+insert into Test
+select element {'id':3,'gid':1,'val':double(4.32),'valplus':double('3.0')};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_mixed/serial_avg_mixed.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_mixed/serial_avg_mixed.3.query.sqlpp
new file mode 100644
index 0000000..e48d5b8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate/serial_avg_mixed/serial_avg_mixed.3.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ /*
+ * Description : Run serial-sum over an ordered list with mixed types
+ * Expected Res : Failure
+ * Date : March 5th 2018
+ */
+
+-- param max-warnings:json=1000
+
+use test;
+
+select gid, strict_avg((select value g.valplus from g)) as avg
+from Test as t
+/* +hash */
+group by t.gid as gid group as g(valplus as valplus);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/get-non-query/get-non-query.1.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/get-non-query/get-non-query.1.get.http
new file mode 100644
index 0000000..bbacf58
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/get-non-query/get-non-query.1.get.http
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description: testing passing a non-query statement using GET
+ * Result: failure
+ */
+# param statement=CREATE DATAVERSE dv1
+/query/service
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/get-non-query/get-non-query.2.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/get-non-query/get-non-query.2.get.http
new file mode 100644
index 0000000..71153c9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/get-non-query/get-non-query.2.get.http
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description: testing passing a non-query statement using GET
+ * Result: failure
+ */
+# param statement=CREATE TYPE t1 AS {id: int}
+# param readonly=true
+/query/service
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/get-non-query/get-non-query.3.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/get-non-query/get-non-query.3.get.http
new file mode 100644
index 0000000..ba3c68e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/get-non-query/get-non-query.3.get.http
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description: testing passing a non-query statement using GET
+ * Result: failure
+ */
+# param statement=CREATE FUNCTION foo(){1}
+# param readonly=false
+/query/service
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/get-query/get-query.1.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/get-query/get-query.1.get.http
new file mode 100644
index 0000000..f17053a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/get-query/get-query.1.get.http
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description: testing passing a query statement using GET
+ * Result: success
+ */
+-- extractresult=true
+# param statement=FROM Metadata.`Dataverse` v WHERE v.DataverseName = 'Metadata' SELECT VALUE v.DataverseName
+/query/service
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/get-query/get-query.2.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/get-query/get-query.2.get.http
new file mode 100644
index 0000000..57dc1ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/get-query/get-query.2.get.http
@@ -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: testing passing a query statement using GET
+ * Result: success
+ */
+-- extractresult=true
+# param statement=FROM Metadata.`Dataverse` v WHERE v.DataverseName = 'Metadata' SELECT VALUE v.DataverseName
+# param readonly=false
+/query/service
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/get-query/get-query.3.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/get-query/get-query.3.get.http
new file mode 100644
index 0000000..14c90a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/get-query/get-query.3.get.http
@@ -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: testing passing a query statement using GET
+ * Result: success
+ */
+-- extractresult=true
+# param statement=FROM Metadata.`Dataverse` v WHERE v.DataverseName = 'Metadata' SELECT VALUE v.DataverseName
+# param readonly=true
+/query/service
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/ignore-body-for-get/ignore-body-for-get.1.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/ignore-body-for-get/ignore-body-for-get.1.get.http
new file mode 100644
index 0000000..ff8b376
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/ignore-body-for-get/ignore-body-for-get.1.get.http
@@ -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: testing that body of GET requests are ignored
+ * Result: failure (missing the required "statement" parameter since it will be ignored)
+ */
+// statuscode 400
+-- requesttype=application/json
+/query/service
+--body={"statement": "SELECT 1;"}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/ignore-body-for-get/ignore-body-for-get.2.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/ignore-body-for-get/ignore-body-for-get.2.get.http
new file mode 100644
index 0000000..79771fb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/ignore-body-for-get/ignore-body-for-get.2.get.http
@@ -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: testing that body of GET requests are ignored
+ * Result: failure (missing the required "statement" parameter since it will be ignored)
+ */
+// statuscode 400
+-- requesttype=application/x-www-form-urlencoded
+/query/service
+--body={"statement": "SELECT 1;"}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/post-non-query/post-non-query.1.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/post-non-query/post-non-query.1.post.http
new file mode 100644
index 0000000..eec1d95
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/post-non-query/post-non-query.1.post.http
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description: testing passing a non-query statement using POST with readonly set to true
+ * Result: failure
+ */
+/query/service
+--body={"statement": "CREATE TYPE t1 AS {id: int};", "readonly": true}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/post-non-query/post-non-query.2.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/post-non-query/post-non-query.2.post.http
new file mode 100644
index 0000000..0fa570b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/api/post-non-query/post-non-query.2.post.http
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description: testing passing a non-query statement using POST with readonly set to false
+ * Result: success
+ */
+-- extractstatus
+/query/service
+--body={"statement": "CREATE TYPE t1 AS {id: int};", "readonly": false}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/csv-parser-001/csv-parser-001.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/csv-parser-001/csv-parser-001.1.ddl.sqlpp
new file mode 100644
index 0000000..65816f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/csv-parser-001/csv-parser-001.1.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+CREATE TYPE t1 AS {f1: string, f2: string, f3: string, f4: string, f5: string};
+CREATE TYPE t2 AS {f1: string, f2: string, f3: string};
+CREATE TYPE t3 AS {f1: int?, f2: boolean, f3: string?};
+CREATE TYPE t4 AS {f1: string, f2: string, f3: string, f4: string};
+
+CREATE EXTERNAL DATASET ds1(t1) USING localfs(("path"="asterix_nc1://data/csv/sample_09.csv"), ("format"="CSV"), ("header"="FALSE"));
+CREATE EXTERNAL DATASET ds2(t2) USING localfs(("path"="asterix_nc1://data/csv/sample_10.csv"), ("format"="Csv"), ("header"="False"));
+CREATE EXTERNAL DATASET ds3(t1) USING localfs(("path"="asterix_nc1://data/csv/sample_11.csv"), ("format"="csv"), ("header"="FALSE"));
+CREATE EXTERNAL DATASET ds4(t3) USING localfs(("path"="asterix_nc1://data/csv/sample_12.csv"), ("format"="csv"), ("header"="True"), ("null"=""));
+CREATE EXTERNAL DATASET ds5(t4) USING localfs(("path"="asterix_nc1://data/csv/sample_13.csv"), ("format"="csv"), ("header"="True"));
+CREATE EXTERNAL DATASET ds6(t4) USING localfs(("path"="asterix_nc1://data/csv/empty_lines.csv"), ("format"="csv"), ("header"="false"));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/csv-parser-001/csv-parser-001.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/csv-parser-001/csv-parser-001.2.query.sqlpp
new file mode 100644
index 0000000..d870372
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/csv-parser-001/csv-parser-001.2.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM ds1 v SELECT VALUE v;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/csv-parser-001/csv-parser-001.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/csv-parser-001/csv-parser-001.3.query.sqlpp
new file mode 100644
index 0000000..64a2f8a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/csv-parser-001/csv-parser-001.3.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM ds2 v SELECT VALUE v;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/csv-parser-001/csv-parser-001.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/csv-parser-001/csv-parser-001.4.query.sqlpp
new file mode 100644
index 0000000..313198c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/csv-parser-001/csv-parser-001.4.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM ds3 v SELECT VALUE v;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/csv-parser-001/csv-parser-001.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/csv-parser-001/csv-parser-001.5.query.sqlpp
new file mode 100644
index 0000000..065de4e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/csv-parser-001/csv-parser-001.5.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM ds4 v SELECT VALUE v;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/csv-parser-001/csv-parser-001.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/csv-parser-001/csv-parser-001.6.query.sqlpp
new file mode 100644
index 0000000..a3d113d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/csv-parser-001/csv-parser-001.6.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM ds5 v SELECT VALUE v ORDER BY v.f1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/csv-parser-001/csv-parser-001.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/csv-parser-001/csv-parser-001.7.query.sqlpp
new file mode 100644
index 0000000..2e5b312
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/csv-parser-001/csv-parser-001.7.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM ds6 v SELECT VALUE v ORDER BY v.f1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/csv-parser-001/csv-parser-001.8.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/csv-parser-001/csv-parser-001.8.ddl.sqlpp
new file mode 100644
index 0000000..86a1b59
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/csv-parser-001/csv-parser-001.8.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/tsv-parser-001/tsv-parser-002.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/tsv-parser-001/tsv-parser-002.1.ddl.sqlpp
new file mode 100644
index 0000000..cabe54b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/tsv-parser-001/tsv-parser-002.1.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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+CREATE TYPE t1 AS {f1: int, f2: int, f3: string, f4: boolean, f5: bigint, f6: double};
+
+CREATE EXTERNAL DATASET ds1(t1) USING localfs(("path"="asterix_nc1://data/tsv/sample_01.tsv"), ("format"="tsv"), ("header"="FALSE"));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/tsv-parser-001/tsv-parser-002.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/tsv-parser-001/tsv-parser-002.2.query.sqlpp
new file mode 100644
index 0000000..d870372
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/tsv-parser-001/tsv-parser-002.2.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM ds1 v SELECT VALUE v;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/tsv-parser-001/tsv-parser-002.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/tsv-parser-001/tsv-parser-002.3.ddl.sqlpp
new file mode 100644
index 0000000..86a1b59
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/csv-tsv-parser/tsv-parser-001/tsv-parser-002.3.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-1/create-dataset-inline-type-1.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-1/create-dataset-inline-type-1.1.ddl.sqlpp
new file mode 100644
index 0000000..55f3604
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-1/create-dataset-inline-type-1.1.ddl.sqlpp
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+/* Metadata test function */
+
+CREATE FUNCTION listMetadata() {
+ SELECT "Dataset" AS en, d.DatasetName, d.DatatypeDataverseName, d.DatatypeName,
+ d.MetatypeDataverseName, d.MetatypeName
+ FROM Metadata.`Dataset` d
+ WHERE d.DataverseName = "test"
+ UNION ALL
+ SELECT "Datatype" AS en, dt.DatatypeName, dt.Derived
+ FROM Metadata.`Datatype` dt
+ WHERE dt.DataverseName = "test"
+ ORDER BY en, DatasetName, DatatypeName
+};
+
+/* Internal datasets */
+
+CREATE DATASET A_Customers_Default_Closed(
+ c_custkey integer not unknown,
+ c_name string not unknown,
+ c_phone string,
+ c_comment string
+) PRIMARY KEY c_custkey ;
+
+CREATE DATASET A_Customers_Closed(
+ c_custkey integer not unknown,
+ c_name string not unknown,
+ c_phone string,
+ c_comment string
+) CLOSED TYPE PRIMARY KEY c_custkey;
+
+CREATE DATASET A_Customers_Open(
+ c_custkey integer not unknown,
+ c_name string not unknown,
+ c_phone string,
+ c_comment string
+) OPEN TYPE PRIMARY KEY c_custkey;
+
+/* External datasets */
+
+CREATE EXTERNAL DATASET B_Orders_Default_Closed(
+ o_orderkey integer not unknown,
+ o_custkey integer not unknown,
+ o_orderstatus string not unknown,
+ o_totalprice double not unknown,
+ o_orderdate string not unknown,
+ o_orderpriority string not unknown,
+ o_clerk string not unknown,
+ o_shippriority integer not unknown,
+ o_comment string
+)
+USING `localfs`
+((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),
+(`input-format`=`text-input-format`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+CREATE EXTERNAL DATASET B_Orders_Closed(
+ o_orderkey integer not unknown,
+ o_custkey integer not unknown,
+ o_orderstatus string not unknown,
+ o_totalprice double not unknown,
+ o_orderdate string not unknown,
+ o_orderpriority string not unknown,
+ o_clerk string not unknown,
+ o_shippriority integer not unknown,
+ o_comment string
+) CLOSED TYPE
+USING `localfs`
+((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),
+(`input-format`=`text-input-format`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+CREATE EXTERNAL DATASET B_Orders_Open(
+ o_orderkey integer not unknown,
+ o_custkey integer not unknown,
+ o_orderstatus string not unknown,
+ o_totalprice double not unknown,
+ o_orderdate string not unknown,
+ o_orderpriority string not unknown,
+ o_clerk string not unknown,
+ o_shippriority integer not unknown,
+ o_comment string
+) OPEN TYPE
+USING `localfs`
+((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),
+(`input-format`=`text-input-format`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+/* Internal datasets with inline META type */
+
+CREATE DATASET C_Customers_Meta_Default_Closed(
+ c_custkey integer not unknown,
+ c_name string not unknown,
+ c_phone string,
+ c_comment string
+)
+WITH META(c_x integer not unknown, c_y integer)
+PRIMARY KEY c_custkey ;
+
+CREATE DATASET C_Customers_Meta_Closed(
+ c_custkey integer not unknown,
+ c_name string not unknown,
+ c_phone string,
+ c_comment string
+) CLOSED TYPE
+WITH META(
+ c_x integer not unknown,
+ c_y integer
+) CLOSED TYPE
+PRIMARY KEY c_custkey;
+
+CREATE DATASET C_Customers_Meta_Open(
+ c_custkey integer not unknown,
+ c_name string not unknown,
+ c_phone string,
+ c_comment string
+) OPEN TYPE
+WITH META(
+ c_x integer not unknown,
+ c_y integer
+) OPEN TYPE
+PRIMARY KEY c_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-1/create-dataset-inline-type-1.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-1/create-dataset-inline-type-1.2.query.sqlpp
new file mode 100644
index 0000000..52a6324
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-1/create-dataset-inline-type-1.2.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+listMetadata();
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-1/create-dataset-inline-type-1.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-1/create-dataset-inline-type-1.3.ddl.sqlpp
new file mode 100644
index 0000000..8a08888
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-1/create-dataset-inline-type-1.3.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+DROP DATASET A_Customers_Default_Closed;
+DROP DATASET A_Customers_Closed;
+DROP DATASET A_Customers_Open;
+DROP DATASET B_Orders_Default_Closed;
+DROP DATASET B_Orders_Closed;
+DROP DATASET B_Orders_Open;
+DROP DATASET C_Customers_Meta_Default_Closed;
+DROP DATASET C_Customers_Meta_Closed;
+DROP DATASET C_Customers_Meta_Open;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-1/create-dataset-inline-type-1.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-1/create-dataset-inline-type-1.4.query.sqlpp
new file mode 100644
index 0000000..ad16400
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-1/create-dataset-inline-type-1.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Test that inline types are deleted when dataset is dropped
+ */
+
+USE test;
+
+array_count(listMetadata());
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-2/create-dataset-inline-type-2.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-2/create-dataset-inline-type-2.1.ddl.sqlpp
new file mode 100644
index 0000000..82548d9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-2/create-dataset-inline-type-2.1.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+/* Prepare: create datasets with inline type */
+
+CREATE DATASET Cust1(
+ c_custkey integer not unknown,
+ c_name string not unknown
+) PRIMARY KEY c_custkey;
+
+CREATE DATASET Cust2(
+ c_custkey integer not unknown,
+ c_name string not unknown,
+ c_phone string
+) PRIMARY KEY c_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-2/create-dataset-inline-type-2.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-2/create-dataset-inline-type-2.2.query.sqlpp
new file mode 100644
index 0000000..00cad73
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-2/create-dataset-inline-type-2.2.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+SELECT count(*) as cnt
+FROM Metadata.`Datatype` dt
+WHERE dt.DataverseName = "test"
+ AND starts_with(dt.DatatypeName, "$d$t$i$Cust")
+ AND dt.Derived.IsAnonymous
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-2/create-dataset-inline-type-2.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-2/create-dataset-inline-type-2.3.ddl.sqlpp
new file mode 100644
index 0000000..81572f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-2/create-dataset-inline-type-2.3.ddl.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/* Create dataset that attempts to use first dataset's type
+ as its item type -> Expect error: unknown type */
+
+USE test;
+
+CREATE DATASET Cust1X(
+ `$d$t$i$Cust1`
+)
+PRIMARY KEY c_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-2/create-dataset-inline-type-2.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-2/create-dataset-inline-type-2.4.ddl.sqlpp
new file mode 100644
index 0000000..7e30198
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-2/create-dataset-inline-type-2.4.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Create dataset that attempts to use first dataset's type
+ * as its meta item type -> Expect error: unknown type
+ */
+
+USE test;
+
+CREATE DATASET Cust2X(
+ c_custkey integer not unknown,
+ c_name string not unknown
+)
+WITH META(`$d$t$i$Cust2`)
+PRIMARY KEY c_custkey;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-2/create-dataset-inline-type-2.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-2/create-dataset-inline-type-2.5.ddl.sqlpp
new file mode 100644
index 0000000..62c0694
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/create-dataset-inline-type-2/create-dataset-inline-type-2.5.ddl.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/* Create dataset that attempts to use unknown type
+ in inline type definition */
+
+USE test;
+
+CREATE DATASET Cust3X(
+ c_custkey integer not unknown,
+ c_name my_unknown_type
+) PRIMARY KEY c_custkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/dataset-and-index-same-dataverse/dataset-and-index-same-dataverse.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/dataset-and-index-same-dataverse/dataset-and-index-same-dataverse.000.ddl.sqlpp
new file mode 100644
index 0000000..0e61805
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/dataset-and-index-same-dataverse/dataset-and-index-same-dataverse.000.ddl.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test.myType if exists;
+create type test.myType as open { id: uuid, f1: int };
+
+drop dataset test.myDataset if exists;
+create dataset test.myDataset(test.myType) primary key id autogenerated;
+
+use Default;
+create index myIndex on test.myDataset(f1);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/dataset-and-index-same-dataverse/dataset-and-index-same-dataverse.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/dataset-and-index-same-dataverse/dataset-and-index-same-dataverse.001.query.sqlpp
new file mode 100644
index 0000000..2cf6a77
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/dataset-and-index-same-dataverse/dataset-and-index-same-dataverse.001.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+select m.DataverseName
+from Metadata.`Index` m
+where m.DatasetName = "myDataset"
+and m.IndexName = "myIndex";
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/dataset-and-index-same-dataverse/dataset-and-index-same-dataverse.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/dataset-and-index-same-dataverse/dataset-and-index-same-dataverse.999.ddl.sqlpp
new file mode 100644
index 0000000..5c1e653
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/dataset-and-index-same-dataverse/dataset-and-index-same-dataverse.999.ddl.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.000.ddl.sqlpp
new file mode 100644
index 0000000..16f4604
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.000.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+-- param max-warnings:json=1000
+
+create dataverse realDataverse;
+create type realDataverse.realType as open { id: uuid };
+create dataset realDataverse.realDataset1(realDataverse.realType) primary key id autogenerated;
+create dataset realDataverse.realDataset2(realDataverse.realType) primary key id autogenerated;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.001.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.001.ddl.sqlpp
new file mode 100644
index 0000000..6d9fcee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.001.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+-- param max-warnings:json=1000
+
+drop type fakeDataverse.fakeType; // fails, error dataverse not found
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.002.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.002.ddl.sqlpp
new file mode 100644
index 0000000..e864dd6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.002.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+-- param max-warnings:json=1000
+
+drop type fakeDataverse.realType; // fails, error dataverse not found
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.003.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.003.ddl.sqlpp
new file mode 100644
index 0000000..1de1496
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.003.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+-- param max-warnings:json=1000
+
+drop type fakeDataverse.realType if exists; // succeeds, warn dataverse not found
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.004.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.004.ddl.sqlpp
new file mode 100644
index 0000000..0d1de03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.004.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+-- param max-warnings:json=1000
+
+drop dataset fakeDataverse.fakeDataset1; // fails, error dataverse not found
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.005.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.005.ddl.sqlpp
new file mode 100644
index 0000000..5abfc5e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.005.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+-- param max-warnings:json=1000
+
+drop dataset fakeDataverse.realDataset1; // fails, error dataverse not found
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.006.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.006.ddl.sqlpp
new file mode 100644
index 0000000..6c7670e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.006.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+-- param max-warnings:json=1000
+
+drop dataset fakeDataverse.fakeDataset1 if exists; // succeeds, warn dataverse not found
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.007.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.007.ddl.sqlpp
new file mode 100644
index 0000000..1a07955
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.007.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+-- param max-warnings:json=1000
+
+drop dataset fakeDataverse.realDataset1 if exists; // succeeds, warn dataverse not found
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.008.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.008.ddl.sqlpp
new file mode 100644
index 0000000..6744022
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.008.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+-- param max-warnings:json=1000
+
+drop dataset realDataverse.fakeDataset1; // fails, error dataset not found
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.009.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.009.ddl.sqlpp
new file mode 100644
index 0000000..d807d1f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.009.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+-- param max-warnings:json=1000
+
+drop dataset realDataverse.realDataset1; // succeeds
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.010.ddl.sqlpp
new file mode 100644
index 0000000..228c6c7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.010.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+-- param max-warnings:json=1000
+
+drop dataset realDataverse.fakeDataset2 if exists; // succeeds, no warning
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.011.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.011.ddl.sqlpp
new file mode 100644
index 0000000..e5be8cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.011.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+-- param max-warnings:json=1000
+
+drop dataset realDataverse.realDataset2 if exists; // succeeds
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.999.ddl.sqlpp
new file mode 100644
index 0000000..304387f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/drop_dataset_invalid_dataverse/test.999.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+-- param max-warnings:json=1000
+
+drop dataverse realDataverse if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataset-name/invalid-dataset-name.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataset-name/invalid-dataset-name.1.ddl.sqlpp
new file mode 100644
index 0000000..f5c4588
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataset-name/invalid-dataset-name.1.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Empty dataset name -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create dataset ``(id int not unknown) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataset-name/invalid-dataset-name.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataset-name/invalid-dataset-name.2.ddl.sqlpp
new file mode 100644
index 0000000..1bc445a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataset-name/invalid-dataset-name.2.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Dataset name starts with a space character -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create dataset ` a`(id int not unknown) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataset-name/invalid-dataset-name.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataset-name/invalid-dataset-name.3.ddl.sqlpp
new file mode 100644
index 0000000..dc10acd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataset-name/invalid-dataset-name.3.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse-name/invalid-dataverse-name.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse-name/invalid-dataverse-name.1.ddl.sqlpp
new file mode 100644
index 0000000..7322b6d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse-name/invalid-dataverse-name.1.ddl.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Empty dataverse name -> Error
+ */
+
+create dataverse ``;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse-name/invalid-dataverse-name.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse-name/invalid-dataverse-name.2.ddl.sqlpp
new file mode 100644
index 0000000..623e40d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse-name/invalid-dataverse-name.2.ddl.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Dataverse name starts with a space character -> Error
+ */
+
+create dataverse ` a`;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.001.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.001.ddl.sqlpp
new file mode 100644
index 0000000..67ff279
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.001.ddl.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+-- param max-warnings:json=1000
+
+drop type fakeDataverse.myType if exists; // success, issue warning
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.002.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.002.ddl.sqlpp
new file mode 100644
index 0000000..0bd6b85
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.002.ddl.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+-- param max-warnings:json=1000
+
+drop dataverse realDataverse if exists;
+create dataverse realDataverse;
+use realDataverse;
+
+drop type realDataverse.myType if exists; // success, no warning
+create type fakeDataverse.myType as open { id: uuid, f1: int }; // fails, error dataverse not found
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.003.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.003.ddl.sqlpp
new file mode 100644
index 0000000..21127f8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.003.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+-- param max-warnings:json=1000
+
+drop dataverse realDataverse if exists;
+create dataverse realDataverse;
+use realDataverse;
+
+drop type realDataverse.myType if exists; // success, no warning
+create type realDataverse.myType as open { id: uuid, f1: int };
+
+drop dataset fakeDataverse.myDataset if exists; // success, issue warning
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.004.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.004.ddl.sqlpp
new file mode 100644
index 0000000..4a820e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.004.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+-- param max-warnings:json=1000
+
+drop dataverse realDataverse if exists;
+create dataverse realDataverse;
+use realDataverse;
+
+drop type realDataverse.myType if exists; // success, no warning
+create type realDataverse.myType as open { id: uuid, f1: int };
+
+drop dataset realDataverse.myDataset if exists; // success
+create dataset fakeDataverse.myDataset(fakeDataverse.myType) primary key id autogenerated; // fail, fake dataverse not found
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.005.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.005.ddl.sqlpp
new file mode 100644
index 0000000..24aad2a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.005.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+-- param max-warnings:json=1000
+
+drop dataverse realDataverse if exists;
+create dataverse realDataverse;
+use realDataverse;
+
+drop type realDataverse.myType if exists; // success, no warning
+create type realDataverse.myType as open { id: uuid, f1: int };
+
+drop dataset realDataverse.myDataset if exists; // success, no warning
+create dataset realDataverse.myDataset(fakeDataverse.myType) primary key id autogenerated; // fail, fakeDataverse not found
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.006.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.006.ddl.sqlpp
new file mode 100644
index 0000000..19f55b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.006.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+-- param max-warnings:json=1000
+
+drop dataverse realDataverse if exists;
+create dataverse realDataverse;
+use realDataverse;
+
+drop type realDataverse.myType if exists; // success
+create type realDataverse.myType as open { id: uuid, f1: int };
+
+drop dataset realDataverse.myDataset if exists; // success
+create dataset fakeDataverse.myDataset(realDataverse.myType) primary key id autogenerated; // fail, fakeDataverse not found
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.007.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.007.ddl.sqlpp
new file mode 100644
index 0000000..89902f1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.007.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+ -- param max-warnings:json=1000
+
+drop dataverse realDataverse if exists;
+create dataverse realDataverse;
+use realDataverse;
+
+drop type realDataverse.myType if exists;
+create type realDataverse.myType as open { id: uuid, f1: int };
+
+drop dataset realDataverse.myDataset if exists;
+create dataset realDataverse.myDataset(realDataverse.myType) primary key id autogenerated;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.008.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.008.ddl.sqlpp
new file mode 100644
index 0000000..2c9da5e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.008.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+ -- param max-warnings:json=1000
+
+drop dataverse realDataverse if exists;
+create dataverse realDataverse;
+use realDataverse;
+
+drop type realDataverse.myType if exists;
+create type realDataverse.myType as open { id: uuid, f1: int };
+
+drop dataset realDataverse.myDataset if exists;
+create dataset realDataverse.myDataset(realDataverse.myType) primary key id autogenerated;
+
+create index myIndex on fakeDataverse.myDataset(id);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.099.ddl.sqlpp
new file mode 100644
index 0000000..5440627
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-dataverse/invalid-dataverse.099.ddl.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse realDataverse if exists;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-name/invalid-feed-name.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-name/invalid-feed-name.1.ddl.sqlpp
new file mode 100644
index 0000000..783a940
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-name/invalid-feed-name.1.ddl.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Empty feed name -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type Tweet as closed
+{
+ id: int64
+};
+
+create dataset Tweets0(Tweet) primary key id;
+
+create feed `` with {
+ "adapter-name" : "socket_adapter",
+ "sockets" : "127.0.0.1:10001",
+ "address-type" : "IP",
+ "type-name" : "Tweet",
+ "format" : "adm"
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-name/invalid-feed-name.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-name/invalid-feed-name.2.ddl.sqlpp
new file mode 100644
index 0000000..6d59d03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-name/invalid-feed-name.2.ddl.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Feed name starts with a space character -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type Tweet as closed
+{
+ id: int64
+};
+
+create dataset Tweets0(Tweet) primary key id;
+
+create feed ` a` with {
+ "adapter-name" : "socket_adapter",
+ "sockets" : "127.0.0.1:10001",
+ "address-type" : "IP",
+ "type-name" : "Tweet",
+ "format" : "adm"
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-name/invalid-feed-name.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-name/invalid-feed-name.3.ddl.sqlpp
new file mode 100644
index 0000000..dc10acd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-name/invalid-feed-name.3.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-policy-name/invalid-feed-policy-name.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-policy-name/invalid-feed-policy-name.1.ddl.sqlpp
new file mode 100644
index 0000000..3a9f7fc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-policy-name/invalid-feed-policy-name.1.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Empty feed policy name -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create ingestion policy `` from path 'data/feed-policy/policy.properties'
+ definition 'someString';
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-policy-name/invalid-feed-policy-name.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-policy-name/invalid-feed-policy-name.2.ddl.sqlpp
new file mode 100644
index 0000000..c5d3179
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-policy-name/invalid-feed-policy-name.2.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Feed policy name starts with a space character -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create ingestion policy ` a` from path 'data/feed-policy/policy.properties'
+ definition 'someString';
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-policy-name/invalid-feed-policy-name.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-policy-name/invalid-feed-policy-name.3.ddl.sqlpp
new file mode 100644
index 0000000..dc10acd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-feed-policy-name/invalid-feed-policy-name.3.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-index-name/invalid-index-name.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-index-name/invalid-index-name.1.ddl.sqlpp
new file mode 100644
index 0000000..ffad79e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-index-name/invalid-index-name.1.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Empty index name -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create dataset t1(id int not unknown, x int not unknown) primary key id;
+
+create index `` on t1(x);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-index-name/invalid-index-name.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-index-name/invalid-index-name.2.ddl.sqlpp
new file mode 100644
index 0000000..6e0bd7d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-index-name/invalid-index-name.2.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Index name starts with a space character -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create dataset t1(id int not unknown, x int not unknown) primary key id;
+
+create index ` a` on t1(x);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-index-name/invalid-index-name.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-index-name/invalid-index-name.3.ddl.sqlpp
new file mode 100644
index 0000000..dc10acd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-index-name/invalid-index-name.3.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-nodegroup-name/invalid-nodegroup-name.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-nodegroup-name/invalid-nodegroup-name.1.ddl.sqlpp
new file mode 100644
index 0000000..ad1de03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-nodegroup-name/invalid-nodegroup-name.1.ddl.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Empty node group name -> Error
+ */
+
+create nodegroup `` if not exists on asterix_nc1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-nodegroup-name/invalid-nodegroup-name.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-nodegroup-name/invalid-nodegroup-name.2.ddl.sqlpp
new file mode 100644
index 0000000..315d287
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-nodegroup-name/invalid-nodegroup-name.2.ddl.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Node group name starts with a space character -> Error
+ */
+
+create nodegroup ` a` if not exists on asterix_nc1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-type-name/invalid-type-name.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-type-name/invalid-type-name.1.ddl.sqlpp
new file mode 100644
index 0000000..ddaac27
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-type-name/invalid-type-name.1.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Empty type name -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type `` as closed {
+ id: int
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-type-name/invalid-type-name.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-type-name/invalid-type-name.2.ddl.sqlpp
new file mode 100644
index 0000000..5ec9f83
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-type-name/invalid-type-name.2.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Type name starts with a space character -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type ` a` as closed {
+ id: int
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-type-name/invalid-type-name.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-type-name/invalid-type-name.3.ddl.sqlpp
new file mode 100644
index 0000000..dc10acd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-type-name/invalid-type-name.3.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-udf-name/invalid-udf-name.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-udf-name/invalid-udf-name.1.ddl.sqlpp
new file mode 100644
index 0000000..0c9b27d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-udf-name/invalid-udf-name.1.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Empty function name -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create function ``() {
+ 1
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-udf-name/invalid-udf-name.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-udf-name/invalid-udf-name.2.ddl.sqlpp
new file mode 100644
index 0000000..fb6e8e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-udf-name/invalid-udf-name.2.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Function name starts with a space character -> Error
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create function ` a`() {
+ 1
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-udf-name/invalid-udf-name.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-udf-name/invalid-udf-name.3.ddl.sqlpp
new file mode 100644
index 0000000..dc10acd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/ddl/invalid-udf-name/invalid-udf-name.3.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/anonymous_no_auth/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/anonymous_no_auth/test.000.ddl.sqlpp
new file mode 100644
index 0000000..13cfe8a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/anonymous_no_auth/test.000.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+CREATE EXTERNAL DATASET test(test) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="json-data/reviews/single-line/json"),
+("format"="json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/anonymous_no_auth/test.001.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/anonymous_no_auth/test.001.ddl.sqlpp
new file mode 100644
index 0000000..b8d0945
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/anonymous_no_auth/test.001.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+CREATE EXTERNAL DATASET test(test) USING S3 (
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="json-data/reviews/single-line/json"),
+("format"="json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/anonymous_no_auth/test.002.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/anonymous_no_auth/test.002.ddl.sqlpp
new file mode 100644
index 0000000..9eda057
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/anonymous_no_auth/test.002.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+CREATE EXTERNAL DATASET test(test) USING S3 (
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="json-data"),
+("format"="json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/anonymous_no_auth/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/anonymous_no_auth/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/anonymous_no_auth/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/bucket-does-not-exist/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/bucket-does-not-exist/test.000.ddl.sqlpp
new file mode 100644
index 0000000..b174162
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/bucket-does-not-exist/test.000.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="bucket-does-not-exist"),
+("definition"="over-1000-objects"),
+("format"="json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/bucket-does-not-exist/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/bucket-does-not-exist/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/bucket-does-not-exist/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.000.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.000.s3bucket.sqlpp
new file mode 100644
index 0000000..dc8b719
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.000.s3bucket.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.
+ */
+// create S3 bucket with data
+playground data_dir data/csv/empty.csv
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.001.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.001.ddl.sqlpp
new file mode 100644
index 0000000..cef7274
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.001.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+DROP TYPE t1 IF EXISTS;
+CREATE TYPE t1 AS {f1: int, f2: int, f3: int, f4: string};
+
+DROP DATASET ds1 IF EXISTS;
+CREATE EXTERNAL DATASET ds1(t1) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="data_dir"),
+("format"="CSV"),
+("header"=true)
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.002.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.002.query.sqlpp
new file mode 100644
index 0000000..26ccfa7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.002.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.
+ */
+// requesttype=application/json
+// param max-warnings:json=100
+USE test;
+
+FROM ds1 v SELECT VALUE v ORDER BY v.f1 ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.003.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.003.s3bucket.sqlpp
new file mode 100644
index 0000000..2b31301
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.003.s3bucket.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.
+ */
+
+playground data_dir data/csv/empty.csv,data/csv/empty.csv,data/csv/empty.csv
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.005.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.005.query.sqlpp
new file mode 100644
index 0000000..26ccfa7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.005.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.
+ */
+// requesttype=application/json
+// param max-warnings:json=100
+USE test;
+
+FROM ds1 v SELECT VALUE v ORDER BY v.f1 ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.006.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.006.s3bucket.sqlpp
new file mode 100644
index 0000000..5b7650a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.006.s3bucket.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.
+ */
+
+playground data_dir data/csv/header/h_one_rec.csv,data/csv/header/h_only.csv,data/csv/header/h_only_with_ln.csv,data/csv/header/h_one_rec_with_ln.csv
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.008.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.008.query.sqlpp
new file mode 100644
index 0000000..26ccfa7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.008.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.
+ */
+// requesttype=application/json
+// param max-warnings:json=100
+USE test;
+
+FROM ds1 v SELECT VALUE v ORDER BY v.f1 ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.009.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.009.s3bucket.sqlpp
new file mode 100644
index 0000000..d7f55e9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.009.s3bucket.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.
+ */
+
+playground data_dir data/csv/header/h_mul_rec.csv,data/csv/empty.csv,data/csv/header/h_only.csv,data/csv/header/h_mul_rec_with_ln.csv
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.011.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.011.query.sqlpp
new file mode 100644
index 0000000..26ccfa7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.011.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.
+ */
+// requesttype=application/json
+// param max-warnings:json=100
+USE test;
+
+FROM ds1 v SELECT VALUE v ORDER BY v.f1 ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.099.ddl.sqlpp
new file mode 100644
index 0000000..36b2bab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-header/query-dataset.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.000.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.000.s3bucket.sqlpp
new file mode 100644
index 0000000..dc8b719
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.000.s3bucket.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.
+ */
+// create S3 bucket with data
+playground data_dir data/csv/empty.csv
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.001.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.001.ddl.sqlpp
new file mode 100644
index 0000000..9adec32
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.001.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+DROP TYPE t1 IF EXISTS;
+CREATE TYPE t1 AS {f1: int, f2: int, f3: int, f4: string};
+
+DROP DATASET ds1 IF EXISTS;
+CREATE EXTERNAL DATASET ds1(t1) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="data_dir"),
+("format"="CSV"),
+("header"="false")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.002.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.002.query.sqlpp
new file mode 100644
index 0000000..26ccfa7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.002.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.
+ */
+// requesttype=application/json
+// param max-warnings:json=100
+USE test;
+
+FROM ds1 v SELECT VALUE v ORDER BY v.f1 ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.003.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.003.s3bucket.sqlpp
new file mode 100644
index 0000000..2b31301
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.003.s3bucket.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.
+ */
+
+playground data_dir data/csv/empty.csv,data/csv/empty.csv,data/csv/empty.csv
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.005.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.005.query.sqlpp
new file mode 100644
index 0000000..26ccfa7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.005.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.
+ */
+// requesttype=application/json
+// param max-warnings:json=100
+USE test;
+
+FROM ds1 v SELECT VALUE v ORDER BY v.f1 ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.006.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.006.s3bucket.sqlpp
new file mode 100644
index 0000000..d42d322
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.006.s3bucket.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.
+ */
+
+playground data_dir data/csv/no_header/no_h_one_rec.csv,data/csv/empty.csv,data/csv/empty_lines.csv,data/csv/no_header/no_h_one_rec_with_ln.csv
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.008.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.008.query.sqlpp
new file mode 100644
index 0000000..26ccfa7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.008.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.
+ */
+// requesttype=application/json
+// param max-warnings:json=100
+USE test;
+
+FROM ds1 v SELECT VALUE v ORDER BY v.f1 ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.009.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.009.s3bucket.sqlpp
new file mode 100644
index 0000000..b9a9796
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.009.s3bucket.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.
+ */
+
+playground data_dir data/csv/no_header/no_h_mul_rec.csv,data/csv/empty.csv,data/csv/no_header/no_h_mul_rec_with_ln.csv
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.011.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.011.query.sqlpp
new file mode 100644
index 0000000..26ccfa7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.011.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.
+ */
+// requesttype=application/json
+// param max-warnings:json=100
+USE test;
+
+FROM ds1 v SELECT VALUE v ORDER BY v.f1 ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.099.ddl.sqlpp
new file mode 100644
index 0000000..36b2bab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-no-header/query-dataset.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.001.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.001.s3bucket.sqlpp
new file mode 100644
index 0000000..132b862
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.001.s3bucket.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.
+ */
+
+playground data_dir data/csv/no_header/no_h_missing_fields.csv
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.002.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.002.ddl.sqlpp
new file mode 100644
index 0000000..6df570c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.002.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+DROP TYPE t1 IF EXISTS;
+DROP TYPE t2 IF EXISTS;
+DROP TYPE t3 IF EXISTS;
+CREATE TYPE t1 AS {f1: int, f2: int, f3: int, f4: string};
+CREATE TYPE t2 AS {f1: bigint, f2: bigint?, f3: double, f4: double?, f5: string, f6: string?, f7: boolean, f8: boolean?};
+CREATE TYPE t3 AS {f1: bigint, f2: string, f3: string};
+
+DROP DATASET ds1 IF EXISTS;
+CREATE EXTERNAL DATASET ds1(t1) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="data_dir"),
+("format"="CSV"),
+("header"="false")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.003.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.003.query.sqlpp
new file mode 100644
index 0000000..26ccfa7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.003.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.
+ */
+// requesttype=application/json
+// param max-warnings:json=100
+USE test;
+
+FROM ds1 v SELECT VALUE v ORDER BY v.f1 ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.004.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.004.s3bucket.sqlpp
new file mode 100644
index 0000000..6aa6fc8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.004.s3bucket.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.
+ */
+
+playground data_dir data/csv/no_header/no_h_no_closing_q.csv,data/csv/no_header/no_h_one_rec.csv
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.006.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.006.query.sqlpp
new file mode 100644
index 0000000..26ccfa7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.006.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.
+ */
+// requesttype=application/json
+// param max-warnings:json=100
+USE test;
+
+FROM ds1 v SELECT VALUE v ORDER BY v.f1 ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.007.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.007.ddl.sqlpp
new file mode 100644
index 0000000..6f96ecf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.007.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+DROP DATASET ds1 IF EXISTS;
+CREATE EXTERNAL DATASET ds1(t1) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="data_dir"),
+("format"="CSV"),
+("header"="false"),
+("redact-warnings"="True")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.008.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.008.query.sqlpp
new file mode 100644
index 0000000..26ccfa7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.008.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.
+ */
+// requesttype=application/json
+// param max-warnings:json=100
+USE test;
+
+FROM ds1 v SELECT VALUE v ORDER BY v.f1 ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.009.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.009.s3bucket.sqlpp
new file mode 100644
index 0000000..f9da983
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.009.s3bucket.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.
+ */
+
+playground data_dir data/csv/header/h_invalid_values.csv
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.010.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.010.ddl.sqlpp
new file mode 100644
index 0000000..47daa38
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.010.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+DROP DATASET ds2 IF EXISTS;
+CREATE EXTERNAL DATASET ds2(t2) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="data_dir"),
+("format"="CSV"),
+("header"="true"),
+("redact-warnings"=True),
+("null"="\\N")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.011.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.011.query.sqlpp
new file mode 100644
index 0000000..e6b24f3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.011.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.
+ */
+// requesttype=application/json
+// param max-warnings:json=100
+USE test;
+
+FROM ds2 v SELECT VALUE v ORDER BY v.f1 ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.012.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.012.ddl.sqlpp
new file mode 100644
index 0000000..4148939
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.012.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+DROP DATASET ds2 IF EXISTS;
+CREATE EXTERNAL DATASET ds2(t2) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="data_dir"),
+("format"="CSV"),
+("header"="true"),
+("null"="\\N")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.013.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.013.query.sqlpp
new file mode 100644
index 0000000..e6b24f3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.013.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.
+ */
+// requesttype=application/json
+// param max-warnings:json=100
+USE test;
+
+FROM ds2 v SELECT VALUE v ORDER BY v.f1 ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.014.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.014.s3bucket.sqlpp
new file mode 100644
index 0000000..5d3989b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.014.s3bucket.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.
+ */
+
+playground data_dir data/csv/error1_line_num.csv
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.015.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.015.ddl.sqlpp
new file mode 100644
index 0000000..75ba5d6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.015.ddl.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+DROP DATASET ds2 IF EXISTS;
+CREATE EXTERNAL DATASET ds2(t3) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="data_dir"),
+("format"="CSV"),
+("header"="false")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.016.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.016.query.sqlpp
new file mode 100644
index 0000000..e6b24f3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.016.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.
+ */
+// requesttype=application/json
+// param max-warnings:json=100
+USE test;
+
+FROM ds2 v SELECT VALUE v ORDER BY v.f1 ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.017.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.017.s3bucket.sqlpp
new file mode 100644
index 0000000..c35d646
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.017.s3bucket.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.
+ */
+
+playground data_dir data/csv/error2_line_num.csv
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.018.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.018.query.sqlpp
new file mode 100644
index 0000000..e6b24f3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.018.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.
+ */
+// requesttype=application/json
+// param max-warnings:json=100
+USE test;
+
+FROM ds2 v SELECT VALUE v ORDER BY v.f1 ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.099.ddl.sqlpp
new file mode 100644
index 0000000..36b2bab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv-warnings/query-dataset.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/csv/query-dataset.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/csv/query-dataset.000.ddl.sqlpp
new file mode 100644
index 0000000..a63f3bc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/csv/query-dataset.000.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+DROP TYPE test IF EXISTS;
+CREATE TYPE test AS {id: int, year: int?, review: string, details: string?};
+
+DROP DATASET test IF EXISTS;
+CREATE EXTERNAL DATASET test(test) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="csv-data/reviews/csv"),
+("format"="Csv"),
+("header"=False),
+("null"="")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/csv/query-dataset.002.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/csv/query-dataset.002.query.sqlpp
new file mode 100644
index 0000000..6e31eb3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/csv/query-dataset.002.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM test SELECT VALUE test ORDER BY id ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/csv/query-dataset.003.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/csv/query-dataset.003.ddl.sqlpp
new file mode 100644
index 0000000..0ff713d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/csv/query-dataset.003.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATASET test IF EXISTS;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/gz/query-dataset.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/gz/query-dataset.000.ddl.sqlpp
new file mode 100644
index 0000000..b8df075
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/gz/query-dataset.000.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+DROP TYPE test IF EXISTS;
+CREATE TYPE test AS {id: int, year: int?, review: string, details: string?};
+
+DROP DATASET test IF EXISTS;
+CREATE EXTERNAL DATASET test(test) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="csv-data/reviews/gz"),
+("format"="Csv"),
+("header"=false),
+("null"="")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/gz/query-dataset.002.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/gz/query-dataset.002.query.sqlpp
new file mode 100644
index 0000000..6e31eb3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/gz/query-dataset.002.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM test SELECT VALUE test ORDER BY id ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/gz/query-dataset.003.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/gz/query-dataset.003.ddl.sqlpp
new file mode 100644
index 0000000..0ff713d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/gz/query-dataset.003.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATASET test IF EXISTS;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/mixed/query-dataset.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/mixed/query-dataset.000.ddl.sqlpp
new file mode 100644
index 0000000..3ed6a3d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/mixed/query-dataset.000.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+DROP TYPE test IF EXISTS;
+CREATE TYPE test AS {id: int, year: int?, review: string, details: string?};
+
+DROP DATASET test IF EXISTS;
+CREATE EXTERNAL DATASET test(test) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="csv-data/reviews/mixed"),
+("format"="Csv"),
+("header"="false"),
+("null"="")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/mixed/query-dataset.002.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/mixed/query-dataset.002.query.sqlpp
new file mode 100644
index 0000000..6e31eb3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/mixed/query-dataset.002.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM test SELECT VALUE test ORDER BY id ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/mixed/query-dataset.003.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/mixed/query-dataset.003.ddl.sqlpp
new file mode 100644
index 0000000..0ff713d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/csv/mixed/query-dataset.003.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATASET test IF EXISTS;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/custom-buffer-size/external_dataset.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/custom-buffer-size/external_dataset.000.ddl.sqlpp
new file mode 100644
index 0000000..22a30ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/custom-buffer-size/external_dataset.000.ddl.sqlpp
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test1 if exists;
+create external dataset test1(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="json-data/reviews/single-line/json"),
+("format"="json"));
+
+drop dataset test2 if exists;
+create external dataset test2(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="json-data/reviews/multi-lines/json"),
+("format"="json"));
+
+drop dataset test3 if exists;
+create external dataset test3(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="json-data/reviews/multi-lines-with-arrays/json"),
+("format"="json"));
+
+drop dataset test4 if exists;
+create external dataset test4(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="json-data/reviews/multi-lines-with-nested-objects/json"),
+("format"="json"));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/custom-buffer-size/external_dataset.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/custom-buffer-size/external_dataset.001.query.sqlpp
new file mode 100644
index 0000000..7c1708a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/custom-buffer-size/external_dataset.001.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;
+set `compiler.externalscanmemory` "8KB";
+select count(*) `count` from test1;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/custom-buffer-size/external_dataset.002.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/custom-buffer-size/external_dataset.002.query.sqlpp
new file mode 100644
index 0000000..d122f42
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/custom-buffer-size/external_dataset.002.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;
+set `compiler.externalscanmemory` "16KB";
+select count(*) `count` from test2;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/custom-buffer-size/external_dataset.003.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/custom-buffer-size/external_dataset.003.query.sqlpp
new file mode 100644
index 0000000..552f943
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/custom-buffer-size/external_dataset.003.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;
+set `compiler.externalscanmemory` "32KB";
+select count(*) `count` from test3;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/custom-buffer-size/external_dataset.004.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/custom-buffer-size/external_dataset.004.query.sqlpp
new file mode 100644
index 0000000..f30af80
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/custom-buffer-size/external_dataset.004.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;
+set `compiler.externalscanmemory` "8KB";
+select value test3 from test3 order by id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/custom-buffer-size/external_dataset.005.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/custom-buffer-size/external_dataset.005.query.sqlpp
new file mode 100644
index 0000000..af6aff0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/custom-buffer-size/external_dataset.005.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;
+set `compiler.externalscanmemory` "16KB";
+select count(*) `count` from test4;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/custom-buffer-size/external_dataset.006.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/custom-buffer-size/external_dataset.006.query.sqlpp
new file mode 100644
index 0000000..854ac80
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/custom-buffer-size/external_dataset.006.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;
+set `compiler.externalscanmemory` "10KB";
+select value test4 from test4 order by id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/custom-buffer-size/external_dataset.999.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/custom-buffer-size/external_dataset.999.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/custom-buffer-size/external_dataset.999.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/definition-does-not-exist/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/definition-does-not-exist/test.000.ddl.sqlpp
new file mode 100644
index 0000000..7a86e69
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/definition-does-not-exist/test.000.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="definition-does-not-exist"),
+("format"="json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/definition-does-not-exist/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/definition-does-not-exist/test.001.query.sqlpp
new file mode 100644
index 0000000..affdb87
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/definition-does-not-exist/test.001.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select count(*) `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/definition-does-not-exist/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/definition-does-not-exist/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/definition-does-not-exist/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/empty-string-definition/external_dataset.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/empty-string-definition/external_dataset.000.ddl.sqlpp
new file mode 100644
index 0000000..a3bfd53
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/empty-string-definition/external_dataset.000.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="fixed-data"),
+("definition"=""),
+("format"="json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/empty-string-definition/external_dataset.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/empty-string-definition/external_dataset.001.query.sqlpp
new file mode 100644
index 0000000..ec40ee0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/empty-string-definition/external_dataset.001.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 count(*) `count` from test;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/empty-string-definition/external_dataset.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/empty-string-definition/external_dataset.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/empty-string-definition/external_dataset.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/bad-name-1/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/bad-name-1/test.000.ddl.sqlpp
new file mode 100644
index 0000000..1b463bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/bad-name-1/test.000.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="json"),
+("exclude"="*.json"),
+("exclude1"="*.json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/bad-name-1/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/bad-name-1/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/bad-name-1/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/bad-name-2/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/bad-name-2/test.000.ddl.sqlpp
new file mode 100644
index 0000000..fe3922d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/bad-name-2/test.000.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="json"),
+("exclude"="*.json"),
+("exclude#"="*.json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/bad-name-2/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/bad-name-2/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/bad-name-2/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/bad-name-3/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/bad-name-3/test.000.ddl.sqlpp
new file mode 100644
index 0000000..efeffc4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/bad-name-3/test.000.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="json"),
+("exclude"="*.json"),
+("exclude#hello"="*.json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/bad-name-3/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/bad-name-3/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/bad-name-3/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/both/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/both/test.000.ddl.sqlpp
new file mode 100644
index 0000000..542046d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/both/test.000.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="json"),
+("include"="*.json"),
+("exclude"="*.json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/both/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/both/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/both/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-1/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-1/test.000.ddl.sqlpp
new file mode 100644
index 0000000..6e84301
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-1/test.000.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="json"),
+("exclude"="*.?sv")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-1/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-1/test.001.query.sqlpp
new file mode 100644
index 0000000..3306d5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-1/test.001.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+select count(*) as `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-1/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-1/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-1/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-2/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-2/test.000.ddl.sqlpp
new file mode 100644
index 0000000..0ce88b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-2/test.000.ddl.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="json"),
+("exclude"="data/mixed/?sv*"),
+("exclude#1"="data/mixed/json/extension*"),
+("exclude#100"="data/mixed/json/EXTENSION*")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-2/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-2/test.001.query.sqlpp
new file mode 100644
index 0000000..3306d5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-2/test.001.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+select count(*) as `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-2/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-2/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-2/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-3/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-3/test.000.ddl.sqlpp
new file mode 100644
index 0000000..8c05fa6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-3/test.000.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="json"),
+("exclude"="data/mixed/?sv*"),
+("exclude#100"="data/mixed/json*201?*")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-3/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-3/test.001.query.sqlpp
new file mode 100644
index 0000000..3306d5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-3/test.001.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+select count(*) as `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-3/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-3/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-3/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-4/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-4/test.000.ddl.sqlpp
new file mode 100644
index 0000000..e1a50b8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-4/test.000.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="json"),
+("exclude"="data/mixed/?sv*"),
+("exclude#1"="data/mixed/json*bye*")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-4/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-4/test.001.query.sqlpp
new file mode 100644
index 0000000..3306d5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-4/test.001.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+select count(*) as `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-4/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-4/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-4/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-5/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-5/test.000.ddl.sqlpp
new file mode 100644
index 0000000..32e115a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-5/test.000.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="json"),
+("exclude"="data/mixed/?sv*"),
+("exclude#1"="data/mixed/json/extension/hello-world-2018.json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-5/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-5/test.001.query.sqlpp
new file mode 100644
index 0000000..3306d5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-5/test.001.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+select count(*) as `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-5/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-5/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-5/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-6/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-6/test.000.ddl.sqlpp
new file mode 100644
index 0000000..78a8310
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-6/test.000.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="json"),
+("exclude"="*.[abct][abcs][abcv]")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-6/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-6/test.001.query.sqlpp
new file mode 100644
index 0000000..3306d5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-6/test.001.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+select count(*) as `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-6/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-6/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-6/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-all/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-all/test.000.ddl.sqlpp
new file mode 100644
index 0000000..f09079a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-all/test.000.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="json"),
+("exclude"="*")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-all/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-all/test.001.query.sqlpp
new file mode 100644
index 0000000..3306d5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-all/test.001.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+select count(*) as `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-all/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-all/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/exclude-all/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-1/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-1/test.000.ddl.sqlpp
new file mode 100644
index 0000000..1b12acb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-1/test.000.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="json"),
+("include"="*2018*.json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-1/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-1/test.001.query.sqlpp
new file mode 100644
index 0000000..3306d5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-1/test.001.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+select count(*) as `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-1/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-1/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-1/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-10/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-10/test.000.ddl.sqlpp
new file mode 100644
index 0000000..32c57b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-10/test.000.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="csv"),
+("header"=false),
+("include"="*[abc][.*")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-10/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-10/test.001.query.sqlpp
new file mode 100644
index 0000000..3306d5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-10/test.001.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+select count(*) as `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-10/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-10/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-10/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-11/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-11/test.000.ddl.sqlpp
new file mode 100644
index 0000000..e4cf69b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-11/test.000.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="csv"),
+("header"=false),
+("include"="*.[a-c][a-z][a-z**||\\\\&&--~~]")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-11/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-11/test.001.query.sqlpp
new file mode 100644
index 0000000..3306d5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-11/test.001.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+select count(*) as `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-11/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-11/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-11/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-12/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-12/test.000.ddl.sqlpp
new file mode 100644
index 0000000..47fbaef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-12/test.000.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+ // This test case matches nothing for "include", but has extreme cases and complicated pattern, expected is to not fail
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="csv"),
+("header"=false),
+("include"="[][!][^]]]]*[![*a-zA--&&^$||0-9B$\\*&&]*&&[^a-b||0--9][[[")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-12/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-12/test.001.query.sqlpp
new file mode 100644
index 0000000..3306d5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-12/test.001.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+select count(*) as `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-12/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-12/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-12/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-2/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-2/test.000.ddl.sqlpp
new file mode 100644
index 0000000..e129135
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-2/test.000.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="json"),
+("include"="*201?*.json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-2/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-2/test.001.query.sqlpp
new file mode 100644
index 0000000..3306d5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-2/test.001.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+select count(*) as `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-2/test.002.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-2/test.002.query.sqlpp
new file mode 100644
index 0000000..9de4a37
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-2/test.002.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+select value test from test order by id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-2/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-2/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-2/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-3/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-3/test.000.ddl.sqlpp
new file mode 100644
index 0000000..4203999
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-3/test.000.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+f1: int,
+f2: string
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="csv"),
+("header"=false),
+("include"="*201?*.csv")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-3/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-3/test.001.query.sqlpp
new file mode 100644
index 0000000..3306d5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-3/test.001.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+select count(*) as `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-3/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-3/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-3/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-4/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-4/test.000.ddl.sqlpp
new file mode 100644
index 0000000..92fed55
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-4/test.000.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+f1: int,
+f2: string
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="tsv"),
+("header"=false),
+("include"="*201?*.tsv")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-4/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-4/test.001.query.sqlpp
new file mode 100644
index 0000000..3306d5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-4/test.001.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+select count(*) as `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-4/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-4/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-4/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-5/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-5/test.000.ddl.sqlpp
new file mode 100644
index 0000000..2d38a8c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-5/test.000.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="json"),
+("include"="*bye*.json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-5/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-5/test.001.query.sqlpp
new file mode 100644
index 0000000..3306d5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-5/test.001.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+select count(*) as `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-5/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-5/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-5/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-6/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-6/test.000.ddl.sqlpp
new file mode 100644
index 0000000..5b4344a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-6/test.000.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="json"),
+("include"="data/mixed/json/*EXTENSION*")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-6/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-6/test.001.query.sqlpp
new file mode 100644
index 0000000..3306d5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-6/test.001.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+select count(*) as `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-6/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-6/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-6/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-7/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-7/test.000.ddl.sqlpp
new file mode 100644
index 0000000..e4e2050
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-7/test.000.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="json"),
+("include"="data/mixed/json/NO-EXTENSION*"),
+("include#0"="data/mixed/json/EXTENSION*")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-7/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-7/test.001.query.sqlpp
new file mode 100644
index 0000000..3306d5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-7/test.001.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+select count(*) as `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-7/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-7/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-7/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-8/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-8/test.000.ddl.sqlpp
new file mode 100644
index 0000000..7d093b2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-8/test.000.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="csv"),
+("header"=false),
+("include"="*.[!xyt][!xyz][!xyz]")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-8/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-8/test.001.query.sqlpp
new file mode 100644
index 0000000..3306d5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-8/test.001.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+select count(*) as `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-8/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-8/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-8/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-9/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-9/test.000.ddl.sqlpp
new file mode 100644
index 0000000..9b0e7ff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-9/test.000.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="csv"),
+("header"=false),
+("include"="*.[a-c][a-z][a-z]")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-9/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-9/test.001.query.sqlpp
new file mode 100644
index 0000000..3306d5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-9/test.001.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+select count(*) as `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-9/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-9/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-9/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-all/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-all/test.000.ddl.sqlpp
new file mode 100644
index 0000000..5545e38
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-all/test.000.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="json"),
+("include"="*")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-all/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-all/test.001.query.sqlpp
new file mode 100644
index 0000000..3306d5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-all/test.001.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+select count(*) as `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-all/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-all/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/include-exclude/include-all/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/invalid-endpoint/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/invalid-endpoint/test.000.ddl.sqlpp
new file mode 100644
index 0000000..da2b945
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/invalid-endpoint/test.000.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="^invalid-endpoint^"),
+("container"="bucket-does-not-exist"),
+("definition"="over-1000-objects"),
+("format"="json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/invalid-endpoint/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/invalid-endpoint/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/invalid-endpoint/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json-warnings/json-warnings.001.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json-warnings/json-warnings.001.s3bucket.sqlpp
new file mode 100644
index 0000000..2bd413b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json-warnings/json-warnings.001.s3bucket.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.
+ */
+
+playground data_dir $$1.json:data/json/malformed-json-no-closing.json,$$2.json:data/json/double-150-11.json
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json-warnings/json-warnings.002.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json-warnings/json-warnings.002.ddl.sqlpp
new file mode 100644
index 0000000..7112bb9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json-warnings/json-warnings.002.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+DROP TYPE t1 IF EXISTS;
+CREATE TYPE t1 AS {};
+
+DROP DATASET ds1 IF EXISTS;
+CREATE EXTERNAL DATASET ds1(t1) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="data_dir"),
+("format"="JSON")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json-warnings/json-warnings.003.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json-warnings/json-warnings.003.query.sqlpp
new file mode 100644
index 0000000..5f2ad26
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json-warnings/json-warnings.003.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.
+ */
+// requesttype=application/json
+// param max-warnings:json=100
+USE test;
+
+FROM ds1 v SELECT VALUE v;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json-warnings/json-warnings.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json-warnings/json-warnings.099.ddl.sqlpp
new file mode 100644
index 0000000..36b2bab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json-warnings/json-warnings.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/gz/external_dataset.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/gz/external_dataset.000.ddl.sqlpp
new file mode 100644
index 0000000..bd6c7ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/gz/external_dataset.000.ddl.sqlpp
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test1 if exists;
+create external dataset test1(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="json-data/reviews/single-line/gz"),
+("format"="json")
+);
+
+drop dataset test2 if exists;
+create external dataset test2(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="json-data/reviews/multi-lines/gz"),
+("format"="json")
+);
+
+drop dataset test3 if exists;
+create external dataset test3(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="json-data/reviews/multi-lines-with-arrays/gz"),
+("format"="json")
+);
+
+drop dataset test4 if exists;
+create external dataset test4(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="json-data/reviews/multi-lines-with-nested-objects/gz"),
+("format"="json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/gz/external_dataset.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/gz/external_dataset.001.query.sqlpp
new file mode 100644
index 0000000..b72e741
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/gz/external_dataset.001.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 count(*) `count` from test1;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/gz/external_dataset.002.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/gz/external_dataset.002.query.sqlpp
new file mode 100644
index 0000000..c3e883e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/gz/external_dataset.002.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 count(*) `count` from test2;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/gz/external_dataset.003.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/gz/external_dataset.003.query.sqlpp
new file mode 100644
index 0000000..e3d64ff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/gz/external_dataset.003.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 count(*) `count` from test3;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/gz/external_dataset.004.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/gz/external_dataset.004.query.sqlpp
new file mode 100644
index 0000000..1fda76a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/gz/external_dataset.004.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 test3 from test3 order by id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/gz/external_dataset.005.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/gz/external_dataset.005.query.sqlpp
new file mode 100644
index 0000000..2b9b5cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/gz/external_dataset.005.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 count(*) `count` from test4;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/gz/external_dataset.006.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/gz/external_dataset.006.query.sqlpp
new file mode 100644
index 0000000..2cd184d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/gz/external_dataset.006.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 test4 from test4 order by id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/gz/external_dataset.007.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/gz/external_dataset.007.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/gz/external_dataset.007.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.000.ddl.sqlpp
new file mode 100644
index 0000000..87a2cef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.000.ddl.sqlpp
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test1 if exists;
+create external dataset test1(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="json-data/reviews/single-line/json"),
+("format"="json")
+);
+
+drop dataset test2 if exists;
+create external dataset test2(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="json-data/reviews/multi-lines/json"),
+("format"="json")
+);
+
+drop dataset test3 if exists;
+create external dataset test3(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="json-data/reviews/multi-lines-with-arrays/json"),
+("format"="json")
+);
+
+drop dataset test4 if exists;
+create external dataset test4(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="json-data/reviews/multi-lines-with-nested-objects/json"),
+("format"="json")
+);
+
+drop dataset test5 if exists;
+create external dataset test5(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="json-data/single-line/json-array-of-objects"),
+("format"="json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.001.query.sqlpp
new file mode 100644
index 0000000..b72e741
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.001.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 count(*) `count` from test1;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.002.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.002.query.sqlpp
new file mode 100644
index 0000000..c3e883e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.002.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 count(*) `count` from test2;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.003.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.003.query.sqlpp
new file mode 100644
index 0000000..e3d64ff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.003.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 count(*) `count` from test3;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.004.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.004.query.sqlpp
new file mode 100644
index 0000000..1fda76a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.004.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 test3 from test3 order by id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.005.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.005.query.sqlpp
new file mode 100644
index 0000000..2b9b5cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.005.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 count(*) `count` from test4;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.006.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.006.query.sqlpp
new file mode 100644
index 0000000..2cd184d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.006.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 test4 from test4 order by id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.007.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.007.query.sqlpp
new file mode 100644
index 0000000..cdde056
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.007.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select value count(*) from test5;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/json/external_dataset.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/mixed/external_dataset.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/mixed/external_dataset.000.ddl.sqlpp
new file mode 100644
index 0000000..32d8388
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/mixed/external_dataset.000.ddl.sqlpp
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test1 if exists;
+create external dataset test1(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="json-data/reviews/single-line/mixed"),
+("format"="json")
+);
+
+drop dataset test2 if exists;
+create external dataset test2(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="json-data/reviews/multi-lines/mixed"),
+("format"="json")
+);
+
+drop dataset test3 if exists;
+create external dataset test3(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="json-data/reviews/multi-lines-with-arrays/mixed"),
+("format"="json")
+);
+
+drop dataset test4 if exists;
+create external dataset test4(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="json-data/reviews/multi-lines-with-nested-objects/mixed"),
+("format"="json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/mixed/external_dataset.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/mixed/external_dataset.001.query.sqlpp
new file mode 100644
index 0000000..b72e741
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/mixed/external_dataset.001.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 count(*) `count` from test1;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/mixed/external_dataset.002.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/mixed/external_dataset.002.query.sqlpp
new file mode 100644
index 0000000..c3e883e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/mixed/external_dataset.002.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 count(*) `count` from test2;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/mixed/external_dataset.003.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/mixed/external_dataset.003.query.sqlpp
new file mode 100644
index 0000000..e3d64ff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/mixed/external_dataset.003.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 count(*) `count` from test3;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/mixed/external_dataset.004.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/mixed/external_dataset.004.query.sqlpp
new file mode 100644
index 0000000..1fda76a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/mixed/external_dataset.004.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 test3 from test3 order by id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/mixed/external_dataset.005.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/mixed/external_dataset.005.query.sqlpp
new file mode 100644
index 0000000..2b9b5cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/mixed/external_dataset.005.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 count(*) `count` from test4;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/mixed/external_dataset.006.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/mixed/external_dataset.006.query.sqlpp
new file mode 100644
index 0000000..2cd184d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/mixed/external_dataset.006.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 test4 from test4 order by id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/mixed/external_dataset.007.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/mixed/external_dataset.007.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/json/mixed/external_dataset.007.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/jsonl/query-dataset.000.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/jsonl/query-dataset.000.s3bucket.sqlpp
new file mode 100644
index 0000000..5def284
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/jsonl/query-dataset.000.s3bucket.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.
+ */
+// create S3 bucket with data
+playground data_dir level1a:data/jsonl/multi-lines/20-records.json,level1a/level2a:data/jsonl/multi-lines-with-arrays/5-records.json,level1b:data/jsonl/multi-lines-with-nested-objects/5-records.json,level1b/level2b:data/jsonl/single-line/20-records.json
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/jsonl/query-dataset.001.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/jsonl/query-dataset.001.ddl.sqlpp
new file mode 100644
index 0000000..75535fb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/jsonl/query-dataset.001.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+DROP TYPE t1 IF EXISTS;
+CREATE TYPE t1 AS OPEN {};
+
+DROP DATASET ds1 IF EXISTS;
+CREATE EXTERNAL DATASET ds1(t1) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="data_dir"),
+("format"="json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/jsonl/query-dataset.002.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/jsonl/query-dataset.002.query.sqlpp
new file mode 100644
index 0000000..87490e6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/jsonl/query-dataset.002.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.
+ */
+// requesttype=application/json
+// param max-warnings:json=100
+USE test;
+
+FROM ds1 v SELECT VALUE v ORDER BY v.id ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/jsonl/query-dataset.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/jsonl/query-dataset.099.ddl.sqlpp
new file mode 100644
index 0000000..36b2bab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/jsonl/query-dataset.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.000.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.000.s3bucket.sqlpp
new file mode 100644
index 0000000..edae2a6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.000.s3bucket.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.
+ */
+// create S3 bucket with data
+playground malformed-data data/json/duplicate-fields.json
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.001.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.001.ddl.sqlpp
new file mode 100644
index 0000000..6ee0509
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.001.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+DROP TYPE t1 IF EXISTS;
+CREATE TYPE t1 AS {};
+
+DROP DATASET ds1 IF EXISTS;
+CREATE EXTERNAL DATASET ds1(t1) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="malformed-data"),
+("format"="json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.002.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.002.query.sqlpp
new file mode 100644
index 0000000..3dedd2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.002.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+USE test;
+
+FROM ds1 v SELECT VALUE v;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.003.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.003.s3bucket.sqlpp
new file mode 100644
index 0000000..ea73e7e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.003.s3bucket.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.
+ */
+// create S3 bucket with data
+playground malformed-data data/json/malformed-json.json
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.004.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.004.query.sqlpp
new file mode 100644
index 0000000..3dedd2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.004.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+USE test;
+
+FROM ds1 v SELECT VALUE v;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.005.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.005.s3bucket.sqlpp
new file mode 100644
index 0000000..ef6d8df
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.005.s3bucket.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.
+ */
+// create S3 bucket with data
+playground malformed-data data/json/malformed-json-2.json
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.006.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.006.query.sqlpp
new file mode 100644
index 0000000..3dedd2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.006.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+USE test;
+
+FROM ds1 v SELECT VALUE v;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.007.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.007.s3bucket.sqlpp
new file mode 100644
index 0000000..3c84eda
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.007.s3bucket.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.
+ */
+// create S3 bucket with data
+playground malformed-data data/jsonl/malformed-jsonl-1.json
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.008.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.008.query.sqlpp
new file mode 100644
index 0000000..3dedd2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.008.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+USE test;
+
+FROM ds1 v SELECT VALUE v;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.009.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.009.s3bucket.sqlpp
new file mode 100644
index 0000000..25f0c8c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.009.s3bucket.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.
+ */
+// create S3 bucket with data
+playground malformed-data data/jsonl/malformed-jsonl-2.json
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.010.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.010.query.sqlpp
new file mode 100644
index 0000000..3dedd2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.010.query.sqlpp
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+USE test;
+
+FROM ds1 v SELECT VALUE v;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.099.ddl.sqlpp
new file mode 100644
index 0000000..36b2bab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/malformed-json/malformed-json.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/negative/negative.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/negative/negative.000.ddl.sqlpp
new file mode 100644
index 0000000..69768e8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/negative/negative.000.ddl.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// "format" parameter is missing for S3
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+DROP TYPE test IF EXISTS;
+CREATE TYPE test AS {id: int, year: int?, review: string, details: string?};
+
+DROP DATASET test IF EXISTS;
+CREATE EXTERNAL DATASET test(test) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="tsv-data/reviews")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/definition-points-to-nothing/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/definition-points-to-nothing/test.000.ddl.sqlpp
new file mode 100644
index 0000000..b51a33f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/definition-points-to-nothing/test.000.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+-- param max-warnings:json=1000
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="point-to-nothing"),
+("format"="json"),
+("include"="*")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/definition-points-to-nothing/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/definition-points-to-nothing/test.001.query.sqlpp
new file mode 100644
index 0000000..284eb58
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/definition-points-to-nothing/test.001.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+-- param max-warnings:json=1000
+
+use test;
+select count(*) as `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/definition-points-to-nothing/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/definition-points-to-nothing/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/definition-points-to-nothing/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/exclude-all-files/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/exclude-all-files/test.000.ddl.sqlpp
new file mode 100644
index 0000000..2e3dc68
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/exclude-all-files/test.000.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+-- param max-warnings:json=1000
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="json"),
+("exclude"="*")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/exclude-all-files/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/exclude-all-files/test.001.query.sqlpp
new file mode 100644
index 0000000..284eb58
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/exclude-all-files/test.001.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+-- param max-warnings:json=1000
+
+use test;
+select count(*) as `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/exclude-all-files/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/exclude-all-files/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/exclude-all-files/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/include-no-files/test.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/include-no-files/test.000.ddl.sqlpp
new file mode 100644
index 0000000..9abdbde
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/include-no-files/test.000.ddl.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+-- param max-warnings:json=1000
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="include-exclude"),
+("definition"="data/mixed/"),
+("format"="json"),
+("include"="*.notRealExtension")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/include-no-files/test.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/include-no-files/test.001.query.sqlpp
new file mode 100644
index 0000000..284eb58
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/include-no-files/test.001.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+-- param max-warnings:json=1000
+
+use test;
+select count(*) as `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/include-no-files/test.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/include-no-files/test.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/no-files-returned/include-no-files/test.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/non-s3-region/external_dataset.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/non-s3-region/external_dataset.000.ddl.sqlpp
new file mode 100644
index 0000000..3b22f11
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/non-s3-region/external_dataset.000.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Creating an external dataset should not fail when non-s3-region is used (some-new-region)
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+ ("accessKeyId"="dummyAccessKey"),
+ ("secretAccessKey"="dummySecretKey"),
+ ("region"="some-new-region"),
+ ("serviceEndpoint"="http://localhost:8001"),
+ ("container"="playground"),
+ ("definition"="json-data/reviews/single-line/json"),
+ ("format"="json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/non-s3-region/external_dataset.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/non-s3-region/external_dataset.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/non-s3-region/external_dataset.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/over-1000-objects/over-1000-objects.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/over-1000-objects/over-1000-objects.000.ddl.sqlpp
new file mode 100644
index 0000000..a26caeb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/over-1000-objects/over-1000-objects.000.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop type test if exists;
+create type test as open {
+};
+
+drop dataset test if exists;
+create external dataset test(test) using S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="over-1000-objects"),
+("format"="json")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/over-1000-objects/over-1000-objects.001.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/over-1000-objects/over-1000-objects.001.query.sqlpp
new file mode 100644
index 0000000..affdb87
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/over-1000-objects/over-1000-objects.001.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select count(*) `count` from test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/over-1000-objects/over-1000-objects.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/over-1000-objects/over-1000-objects.099.ddl.sqlpp
new file mode 100644
index 0000000..548e632
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/over-1000-objects/over-1000-objects.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.000.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.000.s3bucket.sqlpp
new file mode 100644
index 0000000..929442e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.000.s3bucket.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.
+ */
+// create S3 bucket with data
+playground data_dir data/tsv/empty.tsv
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.001.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.001.ddl.sqlpp
new file mode 100644
index 0000000..265c884
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.001.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+DROP TYPE t1 IF EXISTS;
+CREATE TYPE t1 AS {f1: int, f2: int, f3: int, f4: string};
+
+DROP DATASET ds1 IF EXISTS;
+CREATE EXTERNAL DATASET ds1(t1) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="data_dir"),
+("format"="tsv"),
+("header"="true")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.002.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.002.query.sqlpp
new file mode 100644
index 0000000..26ccfa7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.002.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.
+ */
+// requesttype=application/json
+// param max-warnings:json=100
+USE test;
+
+FROM ds1 v SELECT VALUE v ORDER BY v.f1 ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.003.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.003.s3bucket.sqlpp
new file mode 100644
index 0000000..dea81a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.003.s3bucket.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.
+ */
+
+playground data_dir data/tsv/empty.tsv,data/tsv/empty.tsv,data/tsv/empty.tsv
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.005.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.005.query.sqlpp
new file mode 100644
index 0000000..26ccfa7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.005.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.
+ */
+// requesttype=application/json
+// param max-warnings:json=100
+USE test;
+
+FROM ds1 v SELECT VALUE v ORDER BY v.f1 ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.006.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.006.s3bucket.sqlpp
new file mode 100644
index 0000000..bbdba98
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.006.s3bucket.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.
+ */
+
+playground data_dir data/tsv/header/h_one_rec.tsv,data/tsv/header/h_only.tsv,data/tsv/header/h_only_with_ln.tsv,data/tsv/header/h_one_rec_with_ln.tsv
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.008.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.008.query.sqlpp
new file mode 100644
index 0000000..26ccfa7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.008.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.
+ */
+// requesttype=application/json
+// param max-warnings:json=100
+USE test;
+
+FROM ds1 v SELECT VALUE v ORDER BY v.f1 ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.009.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.009.s3bucket.sqlpp
new file mode 100644
index 0000000..a2a20d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.009.s3bucket.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.
+ */
+
+playground data_dir data/tsv/header/h_mul_rec.tsv,data/tsv/empty.tsv,data/tsv/header/h_only.tsv,data/tsv/header/h_mul_rec_with_ln.tsv
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.011.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.011.query.sqlpp
new file mode 100644
index 0000000..26ccfa7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.011.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.
+ */
+// requesttype=application/json
+// param max-warnings:json=100
+USE test;
+
+FROM ds1 v SELECT VALUE v ORDER BY v.f1 ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.099.ddl.sqlpp
new file mode 100644
index 0000000..0ff713d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-header/query-dataset.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATASET test IF EXISTS;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.000.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.000.s3bucket.sqlpp
new file mode 100644
index 0000000..929442e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.000.s3bucket.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.
+ */
+// create S3 bucket with data
+playground data_dir data/tsv/empty.tsv
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.001.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.001.ddl.sqlpp
new file mode 100644
index 0000000..aacf1bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.001.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+DROP TYPE t1 IF EXISTS;
+CREATE TYPE t1 AS {f1: int, f2: int, f3: int, f4: string};
+
+DROP DATASET ds1 IF EXISTS;
+CREATE EXTERNAL DATASET ds1(t1) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="data_dir"),
+("format"="tsv"),
+("header"="false")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.002.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.002.query.sqlpp
new file mode 100644
index 0000000..26ccfa7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.002.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.
+ */
+// requesttype=application/json
+// param max-warnings:json=100
+USE test;
+
+FROM ds1 v SELECT VALUE v ORDER BY v.f1 ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.003.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.003.s3bucket.sqlpp
new file mode 100644
index 0000000..dea81a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.003.s3bucket.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.
+ */
+
+playground data_dir data/tsv/empty.tsv,data/tsv/empty.tsv,data/tsv/empty.tsv
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.005.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.005.query.sqlpp
new file mode 100644
index 0000000..26ccfa7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.005.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.
+ */
+// requesttype=application/json
+// param max-warnings:json=100
+USE test;
+
+FROM ds1 v SELECT VALUE v ORDER BY v.f1 ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.006.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.006.s3bucket.sqlpp
new file mode 100644
index 0000000..2a7fa79
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.006.s3bucket.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.
+ */
+
+playground data_dir data/tsv/no_header/no_h_one_rec.tsv,data/tsv/empty.tsv,data/tsv/empty_lines.tsv,data/tsv/no_header/no_h_one_rec_with_ln.tsv
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.008.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.008.query.sqlpp
new file mode 100644
index 0000000..26ccfa7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.008.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.
+ */
+// requesttype=application/json
+// param max-warnings:json=100
+USE test;
+
+FROM ds1 v SELECT VALUE v ORDER BY v.f1 ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.009.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.009.s3bucket.sqlpp
new file mode 100644
index 0000000..ce3de3c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.009.s3bucket.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.
+ */
+
+playground data_dir data/tsv/no_header/no_h_mul_rec.tsv,data/tsv/empty.tsv,data/tsv/no_header/no_h_mul_rec_with_ln.tsv
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.011.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.011.query.sqlpp
new file mode 100644
index 0000000..26ccfa7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.011.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.
+ */
+// requesttype=application/json
+// param max-warnings:json=100
+USE test;
+
+FROM ds1 v SELECT VALUE v ORDER BY v.f1 ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.099.ddl.sqlpp
new file mode 100644
index 0000000..36b2bab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-no-header/query-dataset.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-warnings/query-dataset.001.s3bucket.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-warnings/query-dataset.001.s3bucket.sqlpp
new file mode 100644
index 0000000..2bd6d52
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-warnings/query-dataset.001.s3bucket.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.
+ */
+
+playground data_dir data/tsv/no_header/no_h_missing_fields.tsv
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-warnings/query-dataset.002.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-warnings/query-dataset.002.ddl.sqlpp
new file mode 100644
index 0000000..aacf1bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-warnings/query-dataset.002.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+DROP TYPE t1 IF EXISTS;
+CREATE TYPE t1 AS {f1: int, f2: int, f3: int, f4: string};
+
+DROP DATASET ds1 IF EXISTS;
+CREATE EXTERNAL DATASET ds1(t1) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="data_dir"),
+("format"="tsv"),
+("header"="false")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-warnings/query-dataset.003.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-warnings/query-dataset.003.query.sqlpp
new file mode 100644
index 0000000..26ccfa7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-warnings/query-dataset.003.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.
+ */
+// requesttype=application/json
+// param max-warnings:json=100
+USE test;
+
+FROM ds1 v SELECT VALUE v ORDER BY v.f1 ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-warnings/query-dataset.099.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-warnings/query-dataset.099.ddl.sqlpp
new file mode 100644
index 0000000..36b2bab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv-warnings/query-dataset.099.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/gz/query-dataset.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/gz/query-dataset.000.ddl.sqlpp
new file mode 100644
index 0000000..439eb0d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/gz/query-dataset.000.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+DROP TYPE test IF EXISTS;
+CREATE TYPE test AS {id: int, year: int?, review: string, details: string?};
+
+DROP DATASET test IF EXISTS;
+CREATE EXTERNAL DATASET test(test) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="tsv-data/reviews/gz"),
+("format"="TSV"),
+("header"="False"),
+("null"="")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/gz/query-dataset.002.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/gz/query-dataset.002.query.sqlpp
new file mode 100644
index 0000000..6e31eb3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/gz/query-dataset.002.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM test SELECT VALUE test ORDER BY id ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/gz/query-dataset.003.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/gz/query-dataset.003.ddl.sqlpp
new file mode 100644
index 0000000..0ff713d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/gz/query-dataset.003.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATASET test IF EXISTS;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/mixed/query-dataset.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/mixed/query-dataset.000.ddl.sqlpp
new file mode 100644
index 0000000..232a6d6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/mixed/query-dataset.000.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+DROP TYPE test IF EXISTS;
+CREATE TYPE test AS {id: int, year: int?, review: string, details: string?};
+
+DROP DATASET test IF EXISTS;
+CREATE EXTERNAL DATASET test(test) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="tsv-data/reviews/mixed"),
+("format"="TSV"),
+("header"="False"),
+("null"="")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/mixed/query-dataset.002.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/mixed/query-dataset.002.query.sqlpp
new file mode 100644
index 0000000..6e31eb3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/mixed/query-dataset.002.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM test SELECT VALUE test ORDER BY id ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/mixed/query-dataset.003.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/mixed/query-dataset.003.ddl.sqlpp
new file mode 100644
index 0000000..0ff713d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/mixed/query-dataset.003.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATASET test IF EXISTS;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/tsv/query-dataset.000.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/tsv/query-dataset.000.ddl.sqlpp
new file mode 100644
index 0000000..7a1c0e1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/tsv/query-dataset.000.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+DROP TYPE test IF EXISTS;
+CREATE TYPE test AS {id: int, year: int?, review: string, details: string?};
+
+DROP DATASET test IF EXISTS;
+CREATE EXTERNAL DATASET test(test) USING S3 (
+("accessKeyId"="dummyAccessKey"),
+("secretAccessKey"="dummySecretKey"),
+("region"="us-west-2"),
+("serviceEndpoint"="http://localhost:8001"),
+("container"="playground"),
+("definition"="tsv-data/reviews/tsv"),
+("format"="TSV"),
+("header"="False"),
+("null"="")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/tsv/query-dataset.002.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/tsv/query-dataset.002.query.sqlpp
new file mode 100644
index 0000000..6e31eb3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/tsv/query-dataset.002.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM test SELECT VALUE test ORDER BY id ASC;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/tsv/query-dataset.003.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/tsv/query-dataset.003.ddl.sqlpp
new file mode 100644
index 0000000..0ff713d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-dataset/aws/s3/tsv/tsv/query-dataset.003.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATASET test IF EXISTS;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-case-01/gby-case-01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-case-01/gby-case-01.1.ddl.sqlpp
new file mode 100644
index 0000000..76552e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-case-01/gby-case-01.1.ddl.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create dataset t1(id integer not unknown) open type primary key id;
+
+create function f1() {
+ select x,
+ case when sum(z)=0 then 0 else sum(y*z)/sum(z) end as res
+ from t1
+ group by x
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-case-01/gby-case-01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-case-01/gby-case-01.2.update.sqlpp
new file mode 100644
index 0000000..9c37cf5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-case-01/gby-case-01.2.update.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into t1
+([
+ {
+ "id": 10,
+ "x": 1,
+ "y": 2,
+ "z": 2
+ },
+ {
+ "id": 11,
+ "x": 1,
+ "y": 4,
+ "z": 4
+ },
+ {
+ "id": 12,
+ "x": 1,
+ "y": 8,
+ "z": 8
+ },
+ {
+ "id": 20,
+ "x": 2,
+ "y": 2,
+ "z": 0
+ },
+ {
+ "id": 21,
+ "x": 2,
+ "y": 4,
+ "z": 0
+ },
+ {
+ "id": 22,
+ "x": 2,
+ "y": 8,
+ "z": 0
+ }
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-case-01/gby-case-01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-case-01/gby-case-01.3.query.sqlpp
new file mode 100644
index 0000000..90a432c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-case-01/gby-case-01.3.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select x,
+ case when sum(z)=0 then 0 else sum(y*z)/sum(z) end as res
+from t1
+group by x
+order by x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-case-01/gby-case-01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-case-01/gby-case-01.4.query.sqlpp
new file mode 100644
index 0000000..6cf2ac9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-case-01/gby-case-01.4.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select x, res
+from f1() f
+order by x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.1.ddl.sqlpp
new file mode 100644
index 0000000..10f5546
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description : Testing group by over cross product (ASTERIXDB-2917)
+ * Expected : SUCCESS
+ */
+
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create dataset t1(id integer not unknown) open type primary key id;
+
+create dataset t2(id integer not unknown) open type primary key id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.2.update.sqlpp
new file mode 100644
index 0000000..7fe3777
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.2.update.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into t1 ([
+ { "id": 1, "x": "a" },
+ { "id": 2, "x": "a" },
+ { "id": 3, "x": "a" },
+ { "id": 4, "x": "b" },
+ { "id": 5, "x": "b" },
+ { "id": 6, "x": "b" }
+]);
+
+insert into t2 ([
+ { "id": 11, "y": "c" },
+ { "id": 12, "y": "c" }
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.3.query.sqlpp
new file mode 100644
index 0000000..34ec706
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-cross-join/gby-cross-join.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select t1.x, count(*) as cnt
+from t2, t1
+group by t1.x
+order by t1.x;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-nested-01/gby-nested-01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-nested-01/gby-nested-01.1.ddl.sqlpp
new file mode 100644
index 0000000..d75bfcd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-nested-01/gby-nested-01.1.ddl.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.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create dataset prd(p_id integer not unknown) open type primary key p_id;
+create dataset sls(s_id integer not unknown) open type primary key s_id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-nested-01/gby-nested-01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-nested-01/gby-nested-01.2.update.sqlpp
new file mode 100644
index 0000000..1fb010e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-nested-01/gby-nested-01.2.update.sqlpp
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into prd
+([
+ {
+ "p_id": 1,
+ "p_name": "PRODUCT_1"
+ },
+ {
+ "p_id": 2,
+ "p_name": "PRODUCT_2"
+ }
+]);
+
+insert into sls
+([
+ {
+ "s_id": 10,
+ "s_pid": 1,
+ "s_year": 2019,
+ "s_city": [
+ { "ct_id": "SFO", "ct_sum": 10, "ct_avg": 5.0 },
+ { "ct_id": "SJC", "ct_sum": 12, "ct_avg": 6.0 },
+ { "ct_id": "RNO", "ct_sum": 14, "ct_avg": 7.0 },
+ { "ct_id": "LAS", "ct_sum": 16, "ct_avg": 8.0 }
+ ],
+ "s_state": [
+ { "st_id": "CA", "st_sum": 22, "st_avg": 5.5 },
+ { "st_id": "NV", "st_sum": 30, "st_avg": 7.5 }
+ ]
+ },
+ {
+ "s_id": 11,
+ "s_pid": 1,
+ "s_year": 2020,
+ "s_city": [
+ { "st_id": "SFO", "ct_sum": 20, "ct_avg": 10.0 },
+ { "st_id": "SJC", "ct_sum": 22, "ct_avg": 11.0 },
+ { "st_id": "RNO", "ct_sum": 24, "ct_avg": 12.0 },
+ { "st_id": "LAS", "ct_sum": 26, "ct_avg": 13.0 }
+ ],
+ "s_state": [
+ { "st_id": "CA", "st_sum": 42, "st_avg": 10.5 },
+ { "st_id": "NV", "st_sum": 50, "st_avg": 12.5 }
+ ]
+ }
+]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-nested-01/gby-nested-01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-nested-01/gby-nested-01.3.query.sqlpp
new file mode 100644
index 0000000..f417383
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/group-by/gby-nested-01/gby-nested-01.3.query.sqlpp
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+SELECT p.p_id, p.p_name,
+ (
+ SELECT c.ct_id,
+ SUM(c.ct_sum) AS ct_sum,
+ SUM(c.ct_avg) AS ct_avg
+ FROM scs, scs.s_city AS c
+ GROUP BY c.ct_id
+ ORDER BY c.ct_id
+ ) AS ct,
+ (
+ SELECT s.st_id,
+ SUM(s.st_sum) AS st_sum,
+ SUM(s.st_avg) AS st_avg
+ FROM scs, scs.s_state AS s
+ GROUP BY s.st_id
+ ORDER BY s.st_id
+ ) AS st
+FROM prd AS p
+LET scs = (
+ SELECT s.s_city, s.s_state
+ FROM sls s
+ WHERE p.p_id = s.s_pid
+)
+ORDER BY p.p_id
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/empty-dataset/empty-dataset.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/empty-dataset/empty-dataset.01.ddl.sqlpp
new file mode 100644
index 0000000..fbe4221
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/empty-dataset/empty-dataset.01.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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+CREATE TYPE openType AS {id: string};
+CREATE DATASET ds1(openType) PRIMARY KEY id;
+CREATE DATASET ds2(openType) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/empty-dataset/empty-dataset.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/empty-dataset/empty-dataset.02.update.sqlpp
new file mode 100644
index 0000000..2a193cd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/empty-dataset/empty-dataset.02.update.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+INSERT INTO ds1 {"id": "1", "f": 3};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/empty-dataset/empty-dataset.03.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/empty-dataset/empty-dataset.03.query.sqlpp
new file mode 100644
index 0000000..4a9afc2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/empty-dataset/empty-dataset.03.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;
+
+FROM ds1 LEFT OUTER JOIN ds2 ON ds1.f > ds2.f
+SELECT ds1, ds2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/empty-dataset/empty-dataset.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/empty-dataset/empty-dataset.04.query.sqlpp
new file mode 100644
index 0000000..954341c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/empty-dataset/empty-dataset.04.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;
+
+FROM ds2 LEFT OUTER JOIN ds1 ON ds2.f > ds1.f
+SELECT ds1, ds2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/empty-dataset/empty-dataset.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/empty-dataset/empty-dataset.05.query.sqlpp
new file mode 100644
index 0000000..e5ba174
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/empty-dataset/empty-dataset.05.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;
+
+FROM ds1 JOIN ds2 ON ds1.f > ds2.f
+SELECT ds1, ds2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/empty-dataset/empty-dataset.06.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/empty-dataset/empty-dataset.06.query.sqlpp
new file mode 100644
index 0000000..9e6c335
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/empty-dataset/empty-dataset.06.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;
+
+FROM ds2 JOIN ds1 ON ds2.f > ds1.f
+SELECT ds1, ds2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-02-push-select/loj-02-push-select.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-02-push-select/loj-02-push-select.01.ddl.sqlpp
new file mode 100644
index 0000000..e656d9d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-02-push-select/loj-02-push-select.01.ddl.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type test.TestType as
+{
+ id : integer
+};
+
+create dataset t1(TestType) primary key id;
+
+create dataset t2(TestType) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-02-push-select/loj-02-push-select.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-02-push-select/loj-02-push-select.02.update.sqlpp
new file mode 100644
index 0000000..950f728
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-02-push-select/loj-02-push-select.02.update.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into t1
+([
+ {"id":1, "x":1, "a": 11 },
+ {"id":2, "x":2, "a": 12 },
+ {"id":3, "x":3, "a": 13 },
+ {"id":4, "x":4, "a": 14 }
+]);
+
+insert into t2
+([
+ {"id":1, "y":1, "b": 111 },
+ {"id":2, "y":2, "b": 112 }
+]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-02-push-select/loj-02-push-select.03.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-02-push-select/loj-02-push-select.03.query.sqlpp
new file mode 100644
index 0000000..5be33b9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-02-push-select/loj-02-push-select.03.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Test WHERE .. IS MISSING using right side of an outer join.
+ * 1:1 left outer join.
+ * Expect only 2 tuples in the result (x=3 and x=4 that didn't join)
+ * because t2_right.y evaluates to MISSING in those tuples.
+ */
+
+use test;
+
+select *
+from t1 t1_left left join t2 t2_right on t1_left.x = t2_right.y
+where t2_right.y is missing
+order by t1_left.a;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-02-push-select/loj-02-push-select.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-02-push-select/loj-02-push-select.04.query.sqlpp
new file mode 100644
index 0000000..20ebff4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-02-push-select/loj-02-push-select.04.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Test WHERE .. IS UNKNOWN using right side of an outer join.
+ * 1:1 left outer join.
+ * Expect only 2 tuples in the result (x=3 and x=4 that didn't join)
+ * because t2_right.y evaluates to MISSING in those tuples,
+ * therefore IS UNKNOWN evaluates to TRUE
+ */
+
+use test;
+
+select *
+from t1 t1_left left join t2 t2_right on t1_left.x = t2_right.y
+where t2_right.y is unknown
+order by t1_left.a;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-02-push-select/loj-02-push-select.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-02-push-select/loj-02-push-select.05.query.sqlpp
new file mode 100644
index 0000000..0d9515b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-02-push-select/loj-02-push-select.05.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Test WHERE IF_MISSING(.., "missing") using right side of an outer join.
+ * 1:1 left outer join.
+ * Expect only 2 tuples in the result (x=3 and x=4 that didn't join)
+ * because t2_right.y evaluates to MISSING in those tuples,
+ * therefore IF_MISSING(.., "missing") evaluates to "missing"
+ */
+
+use test;
+
+select *
+from t1 t1_left left join t2 t2_right on t1_left.x = t2_right.y
+where if_missing(t2_right.y, "missing") = "missing"
+order by t1_left.a;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-02-push-select/loj-02-push-select.06.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-02-push-select/loj-02-push-select.06.query.sqlpp
new file mode 100644
index 0000000..9315032
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-02-push-select/loj-02-push-select.06.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Test WHERE clause using both sides of an outer join.
+ * 1:1 left outer join.
+ * Expect only 2 tuples in the result (x=1 and x=2 that did join)
+ * because t2_right.y and therefore to_string(t2_right.y)
+ * evaluate to MISSING for the other 2 that didn't join.
+ */
+
+use test;
+
+select *
+from t1 t1_left left join t2 t2_right on t1_left.x = t2_right.y
+where to_string(t1_left.x) = to_string(t2_right.y)
+order by t1_left.a;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-03-no-listify/loj-03-no-listify.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-03-no-listify/loj-03-no-listify.1.ddl.sqlpp
new file mode 100644
index 0000000..bc81bff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-03-no-listify/loj-03-no-listify.1.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+CREATE TYPE tasksType AS {
+ id : integer
+};
+
+CREATE DATASET tasks(tasksType) PRIMARY KEY id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-03-no-listify/loj-03-no-listify.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-03-no-listify/loj-03-no-listify.2.update.sqlpp
new file mode 100644
index 0000000..061d357
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-03-no-listify/loj-03-no-listify.2.update.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+INSERT INTO tasks
+([
+ {"id": 1, "taskId":1, "status":"x"},
+ {"id": 2, "taskId":1, "status":"x"},
+ {"id": 3, "taskId":1, "status":"y"},
+ {"id": 4, "taskId":1, "status":"y"},
+ {"id": 5, "taskId":1, "status":"z"},
+ {"id": 6, "taskId":1, "status":"z"},
+ {"id": 7, "taskId":2, "status":"x"},
+ {"id": 8, "taskId":2, "status":"x"},
+ {"id": 9, "taskId":2, "status":"y"},
+ {"id": 10, "taskId":2, "status":"y"},
+ {"id": 11, "taskId":2, "status":"z"},
+ {"id": 12, "taskId":2, "status":"z"}
+]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-03-no-listify/loj-03-no-listify.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-03-no-listify/loj-03-no-listify.3.query.sqlpp
new file mode 100644
index 0000000..d64e711
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/loj-03-no-listify/loj-03-no-listify.3.query.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: Test that listify() is eliminated
+ * on the right side of an outer join
+ */
+
+USE test;
+
+SELECT t0.taskId, t0.cnt_all, t1.cnt_x, t2.cnt_y, t3.cnt_z
+FROM (
+ SELECT taskId, COUNT(1) AS cnt_all FROM tasks GROUP BY taskId ORDER BY taskId
+) AS t0
+LEFT OUTER JOIN (
+ SELECT taskId, COUNT(1) AS cnt_x FROM tasks WHERE status="x" GROUP BY taskId
+) AS t1 ON t0.taskId = t1.taskId
+LEFT OUTER JOIN (
+ SELECT taskId, COUNT(1) AS cnt_y FROM tasks WHERE status="y" GROUP BY taskId
+) AS t2 ON t0.taskId = t2.taskId
+LEFT OUTER JOIN (
+ SELECT taskId, COUNT(1) AS cnt_z FROM tasks WHERE status="z" GROUP BY taskId
+) AS t3 ON t0.taskId = t3.taskId
+ORDER BY t0.taskId;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/query-ASTERIXDB-2857/query-ASTERIXDB-2857.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/query-ASTERIXDB-2857/query-ASTERIXDB-2857.1.ddl.sqlpp
new file mode 100644
index 0000000..87b5d75
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/query-ASTERIXDB-2857/query-ASTERIXDB-2857.1.ddl.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Test nested loop implementation of left outer join
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type tenkType as closed {
+ unique1 : integer,
+ unique2 : integer,
+ two : integer,
+ four : integer,
+ ten : integer,
+ twenty : integer,
+ hundred : integer,
+ thousand : integer,
+ twothousand : integer,
+ fivethous : integer,
+ tenthous : integer,
+ odd100 : integer,
+ even100 : integer,
+ stringu1 : string,
+ stringu2 : string,
+ string4 : string
+};
+
+create dataset tenk(tenkType) primary key unique2;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/query-ASTERIXDB-2857/query-ASTERIXDB-2857.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/query-ASTERIXDB-2857/query-ASTERIXDB-2857.2.update.sqlpp
new file mode 100644
index 0000000..2d7e768
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/query-ASTERIXDB-2857/query-ASTERIXDB-2857.2.update.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+load dataset tenk using localfs ((`path`=`asterix_nc1://data/tenk.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/query-ASTERIXDB-2857/query-ASTERIXDB-2857.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/query-ASTERIXDB-2857/query-ASTERIXDB-2857.3.query.sqlpp
new file mode 100644
index 0000000..823a540
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/query-ASTERIXDB-2857/query-ASTERIXDB-2857.3.query.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Test nested loop implementation of left outer join
+ */
+
+USE test;
+
+SELECT
+ t0.unique1 AS t0_unique1,
+ t1.unique1 AS t1_unique1,
+ t2.unique1 AS t2_unique1
+FROM (
+ SELECT unique1, unique2 FROM tenk WHERE unique2 < 2
+) t0
+INNER JOIN (
+ SELECT unique1, unique2 FROM tenk WHERE unique2 < 4
+) t1 ON t0.unique2 = t1.unique2
+LEFT JOIN (
+ SELECT unique1, unique2 FROM tenk WHERE unique2 < 6
+) t2 ON t0.unique2 + t2.unique2 = 2 * t1.unique2
+ORDER BY t0_unique1, t1_unique1, t2_unique1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/right_branch_opt_1/right_branch_opt_1.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/right_branch_opt_1/right_branch_opt_1.1.query.sqlpp
new file mode 100644
index 0000000..016b065
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/leftouterjoin/right_branch_opt_1/right_branch_opt_1.1.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+with
+ t1 as (from range(1, 4) v select v, -v nv, {} as x),
+ t2 as (from range(1, 2) v select v, -v nv, {} as x)
+from t1 left outer join t2 on t1.v = t2.v
+select t1.nv t1nv, t2.nv t2nv, t1.x as t1x, t2.x as t2x
+order by t1nv desc;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.1.ddl.sqlpp
index 917715e..319db54 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.1.ddl.sqlpp
@@ -42,6 +42,28 @@
tweetid : string
};
+create type onekType1 as
+ closed {
+ unique1 : bigint,
+ unique2 : bigint,
+ two : bigint,
+ four : bigint,
+ ten : bigint,
+ twenty : bigint,
+ onePercent : bigint,
+ tenPercent : bigint,
+ twentyPercent : bigint,
+ fiftyPercent : bigint,
+ unique3 : bigint,
+ evenOnePercent : bigint,
+ oddOnePercent : bigint,
+ stringu1 : string,
+ stringu2 : string,
+ string4 : string
+};
+
create dataset DBLP1(DBLPType) primary key id;
-create dataset TweetMessages(TweetMessageType) primary key tweetid;
\ No newline at end of file
+create dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create dataset onek1(onekType1) primary key unique2;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.10.query.sqlpp
new file mode 100644
index 0000000..a91cece
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.10.query.sqlpp
@@ -0,0 +1,46 @@
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Check that data-scan operator isomorphism comparison
+ * considers pushed condition.
+ *
+ * The dataset contains 100 tuples, 50 with two=0 and 50 with two=1.
+ * The expected result is 50.
+ */
+
+use test;
+
+with T1 as (
+ select two from onek1
+ where two between 1 and 10
+ limit 1000
+),
+T2 as (
+ select two from onek1
+ where two between -10 and -1
+ limit 1000
+)
+
+select value count(*) from (
+ select two from T1
+ union all
+ select two from T2
+) t;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.11.query.sqlpp
new file mode 100644
index 0000000..34eb190
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.11.query.sqlpp
@@ -0,0 +1,45 @@
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Check that data-scan operator isomorphism comparison
+ * considers pushed condition
+ */
+
+use test;
+
+explain
+
+with T1 as (
+ select two from onek1
+ where two between 1 and 10
+ limit 1000
+),
+T2 as (
+ select two from onek1
+ where two between -10 and -1
+ limit 1000
+)
+
+select value count(*) from (
+ select two from T1
+ union all
+ select two from T2
+) t;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.2.update.sqlpp
index a18f3e2..cd3d45f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.2.update.sqlpp
@@ -16,11 +16,11 @@
* specific language governing permissions and limitations
* under the License.
*/
-/* scan and print a delimited text file */
use test;
-
load dataset DBLP1 using localfs ((`path`=`asterix_nc1://data/dblp-small/dblp-small-id.txt`),(`format`=`delimited-text`),(`delimiter`=`:`));
-load dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
\ No newline at end of file
+load dataset TweetMessages using localfs ((`path`=`asterix_nc1://data/tinysocial/twm.adm`),(`format`=`adm`));
+
+load dataset onek1 using localfs ((`path`=`asterix_nc1://data/wisc/onektup.adm`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.1.ddl.sqlpp
index 162cc35..36af900 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.1.ddl.sqlpp
@@ -38,5 +38,26 @@
misc : string
};
+create type onekType1 as
+ closed {
+ unique1 : bigint,
+ unique2 : bigint,
+ two : bigint,
+ four : bigint,
+ ten : bigint,
+ twenty : bigint,
+ onePercent : bigint,
+ tenPercent : bigint,
+ twentyPercent : bigint,
+ fiftyPercent : bigint,
+ unique3 : bigint,
+ evenOnePercent : bigint,
+ oddOnePercent : bigint,
+ stringu1 : string,
+ stringu2 : string,
+ string4 : string
+};
+
create dataset DBLP1(DBLPType) primary key id;
+create dataset onek1(onekType1) primary key unique2;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.2.update.sqlpp
index 2e0f056..4d4065d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.2.update.sqlpp
@@ -20,5 +20,6 @@
use test;
-
load dataset DBLP1 using localfs ((`path`=`asterix_nc1://data/dblp-small/dblp-small-id.txt`),(`format`=`delimited-text`),(`delimiter`=`:`));
+
+load dataset onek1 using localfs ((`path`=`asterix_nc1://data/wisc/onektup.adm`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.6.query.sqlpp
new file mode 100644
index 0000000..dd9eb15
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.6.query.sqlpp
@@ -0,0 +1,45 @@
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Check that data-scan operator isomorphism comparison
+ * considers pushed limit.
+ *
+ * The dataset contains 100 tuples,
+ * so the expected result is 110.
+ */
+
+use test;
+
+with
+T1 as (
+ select two from onek1
+ limit 100
+),
+T2 as (
+ select two from onek1
+ limit 10
+)
+
+select value count(*) from (
+ select two from T1
+ union all
+ select two from T2
+) t;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.7.query.sqlpp
new file mode 100644
index 0000000..a592faa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.7.query.sqlpp
@@ -0,0 +1,44 @@
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Check that data-scan operator isomorphism comparison
+ * considers pushed limit.
+ */
+
+use test;
+
+explain
+
+with
+T1 as (
+ select two from onek1
+ limit 100
+),
+T2 as (
+ select two from onek1
+ limit 10
+)
+
+select value count(*) from (
+ select two from T1
+ union all
+ select two from T2
+) t;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/get-item_03/get-item_03.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/get-item_03/get-item_03.4.query.sqlpp
new file mode 100644
index 0000000..ab27331
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/get-item_03/get-item_03.4.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+{
+"ta1": [1, 2, 3][-1],
+"ta2": [1, 2, 3][-2],
+"ta3": [1, 2, 3][-3],
+"ta4": [1, 2, 3][-4] is missing,
+"tb1": [1, 2, 3][-((select value id from test where id = 1)[0])],
+"tb2": [1, 2, 3][-((select value id from test where id = 1)[0])-1],
+"tb3": [1, 2, 3][-((select value id from test where id = 1)[0])-2],
+"tb4": [1, 2, 3][-((select value id from test where id = 1)[0])-3] is missing
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/var-in-list/var-in-list.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/var-in-list/var-in-list.01.ddl.sqlpp
new file mode 100644
index 0000000..5b2b740
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/var-in-list/var-in-list.01.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+CREATE TYPE t1 AS {id: int};
+CREATE DATASET ds1(t1) primary key id;
+
+CREATE FUNCTION fun1(r) {
+ (SELECT VALUE x FROM [r] AS x)[0]
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/var-in-list/var-in-list.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/var-in-list/var-in-list.02.update.sqlpp
new file mode 100644
index 0000000..7535183
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/var-in-list/var-in-list.02.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+INSERT INTO ds1 {"id": 1, "f1": "foo"};
+INSERT INTO ds1 {"id": 2, "f1": "bar"};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/var-in-list/var-in-list.03.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/var-in-list/var-in-list.03.query.sqlpp
new file mode 100644
index 0000000..b9a79b6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/var-in-list/var-in-list.03.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM ds1 AS v SELECT [v] AS list ORDER BY list;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/var-in-list/var-in-list.04.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/var-in-list/var-in-list.04.query.sqlpp
new file mode 100644
index 0000000..d61e5be
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/var-in-list/var-in-list.04.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM ds1 AS s SELECT fun1(s) AS list ORDER BY list;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/var-in-list/var-in-list.05.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/var-in-list/var-in-list.05.query.sqlpp
new file mode 100644
index 0000000..42c7031
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/var-in-list/var-in-list.05.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM [{'a':5}, {'a':7}] AS s SELECT fun1(s) AS list ORDER BY list;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/var-in-list/var-in-list.06.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/var-in-list/var-in-list.06.query.sqlpp
new file mode 100644
index 0000000..924b4b8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/var-in-list/var-in-list.06.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM [{'a':5}, {'b':7}] AS s SELECT fun1(s) AS list ORDER BY list;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/csv_01/csv_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/csv_01/csv_01.2.update.sqlpp
index 988ebe3..68faf3f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/csv_01/csv_01.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/csv_01/csv_01.2.update.sqlpp
@@ -26,5 +26,5 @@
use temp;
-load dataset testds using localfs ((`path`=`asterix_nc1://data/csv/sample_01.csv`),(`format`=`delimited-text`),(`delimiter`=`,`));
+load dataset testds using localfs ((`path`=`asterix_nc1://data/csv/sample_01.csv`),(`format`=`delimited-text`),(`delimiter`=`,`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/csv_02/csv_02.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/csv_02/csv_02.2.update.sqlpp
index 988ebe3..68faf3f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/csv_02/csv_02.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/csv_02/csv_02.2.update.sqlpp
@@ -26,5 +26,5 @@
use temp;
-load dataset testds using localfs ((`path`=`asterix_nc1://data/csv/sample_01.csv`),(`format`=`delimited-text`),(`delimiter`=`,`));
+load dataset testds using localfs ((`path`=`asterix_nc1://data/csv/sample_01.csv`),(`format`=`delimited-text`),(`delimiter`=`,`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/csv_03/csv_03.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/csv_03/csv_03.2.update.sqlpp
index 7eaaf1d..fdc1d1a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/csv_03/csv_03.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/csv_03/csv_03.2.update.sqlpp
@@ -26,5 +26,5 @@
use temp;
-load dataset testds using localfs ((`path`=`asterix_nc1://data/csv/sample_02.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset testds using localfs ((`path`=`asterix_nc1://data/csv/sample_02.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/csv_08_header_cr/csv_08.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/csv_08_header_cr/csv_08.2.update.sqlpp
index e44dfe9..d05c821 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/csv_08_header_cr/csv_08.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/csv_08_header_cr/csv_08.2.update.sqlpp
@@ -26,5 +26,5 @@
use temp;
-load dataset testds using localfs ((`path`=`asterix_nc1://data/csv/sample_08_header.csv.cr`),(`format`=`delimited-text`),(`header`=`true`));
+load dataset testds using localfs ((`path`=`asterix_nc1://data/csv/sample_08_header.csv.cr`),(`format`=`delimited-text`),(`header`=`true`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/csv_08_header_crlf/csv_08.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/csv_08_header_crlf/csv_08.2.update.sqlpp
index e4da86b..9be6773 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/csv_08_header_crlf/csv_08.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/csv_08_header_crlf/csv_08.2.update.sqlpp
@@ -26,5 +26,5 @@
use temp;
-load dataset testds using localfs ((`path`=`asterix_nc1://data/csv/sample_08_header.csv.crlf`),(`format`=`delimited-text`),(`header`=`true`));
+load dataset testds using localfs ((`path`=`asterix_nc1://data/csv/sample_08_header.csv.crlf`),(`format`=`delimited-text`),(`header`=`true`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/csv_08_header_lf/csv_08.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/csv_08_header_lf/csv_08.2.update.sqlpp
index e024bd3..102b2aa 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/csv_08_header_lf/csv_08.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/load/csv_08_header_lf/csv_08.2.update.sqlpp
@@ -26,5 +26,5 @@
use temp;
-load dataset testds using localfs ((`path`=`asterix_nc1://data/csv/sample_08_header.csv.lf`),(`format`=`delimited-text`),(`header`=`true`));
+load dataset testds using localfs ((`path`=`asterix_nc1://data/csv/sample_08_header.csv.lf`),(`format`=`delimited-text`),(`header`=`true`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/external-dataset/external-dataset.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/external-dataset/external-dataset.1.ddl.sqlpp
new file mode 100644
index 0000000..a105dfe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/external-dataset/external-dataset.1.ddl.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Processed objects metrics on external dataset
+ * Expected Res : Success
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+CREATE TYPE t1 AS {f1: string, f2: string, f3: string, f4: string, f5: string};
+
+CREATE EXTERNAL DATASET ds1(t1) USING localfs(("path"="asterix_nc1://data/csv/sample_09.csv"), ("format"="CSV"), ("header"="FALSE"));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/external-dataset/external-dataset.2.metrics.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/external-dataset/external-dataset.2.metrics.sqlpp
new file mode 100644
index 0000000..e57e938
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/external-dataset/external-dataset.2.metrics.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 : Processed objects metrics on external dataset
+ * Expected Res : Success
+ */
+
+USE test;
+
+SELECT COUNT(*) from ds1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/external-dataset/external-datasett.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/external-dataset/external-datasett.3.ddl.sqlpp
new file mode 100644
index 0000000..0bf95e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/metrics/external-dataset/external-datasett.3.ddl.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Processed objects metrics on external dataset
+ * Expected Res : Success
+ */
+
+DROP DATAVERSE test;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_01/case_01.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_01/case_01.1.query.sqlpp
index c2a4cea..7f9647b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_01/case_01.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_01/case_01.1.query.sqlpp
@@ -24,5 +24,5 @@
WHEN MISSING THEN -2
ELSE 2.0/t
END
-FROM [0, 1, 2, 4, NULL, [0][-1]] t;
+FROM [0, 1, 2, 4, NULL, MISSING] t;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_02/case_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_02/case_02.1.query.sqlpp
index ab19bcb..095eb20 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_02/case_02.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_02/case_02.1.query.sqlpp
@@ -24,6 +24,6 @@
WHEN t IS MISSING THEN -2
ELSE 2.0/t
END
-FROM [0, 1, 2, 4, NULL, [0][-1]] t;
+FROM [0, 1, 2, 4, NULL, MISSING] t;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_03/case_03.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_03/case_03.1.query.sqlpp
index 7c3e566..12cd3f6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_03/case_03.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_03/case_03.1.query.sqlpp
@@ -24,5 +24,5 @@
WHEN t IS MISSING THEN (SELECT -2 AS r)
ELSE (SELECT -3 AS r)
END
-FROM [0, 1, 2, 4, NULL, [0][-1]] t;
+FROM [0, 1, 2, 4, NULL, MISSING] t;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_04/case_04.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_04/case_04.1.query.sqlpp
index 154e611..e99da85 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_04/case_04.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_04/case_04.1.query.sqlpp
@@ -24,5 +24,5 @@
WHEN t IS MISSING THEN (SELECT -2)
ELSE 2.0/t
END
-FROM [0, 1, 2, 4, NULL, [0][-1]] t;
+FROM [0, 1, 2, 4, NULL, MISSING] t;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_05/case_05.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_05/case_05.1.query.sqlpp
index 32030a7..5940b19 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_05/case_05.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_05/case_05.1.query.sqlpp
@@ -26,5 +26,5 @@
WHEN t=2 THEN MISSING
ELSE 2.0/t
END
-FROM [0, 1, 2, 4, NULL, [0][-1]] t;
+FROM [0, 1, 2, 4, NULL, MISSING] t;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_06/case_06.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_06/case_06.1.query.sqlpp
index 5bf2786..8fbaa7a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_06/case_06.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_06/case_06.1.query.sqlpp
@@ -22,5 +22,5 @@
WHEN t = 0 THEN MISSING
ELSE NULL
END
-FROM [0, 1, 2, 4, NULL, [0][-1]] t;
+FROM [0, 1, 2, 4, NULL, MISSING] t;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_07/case_07.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_07/case_07.1.query.sqlpp
index 4850047..27180b5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_07/case_07.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/case_07/case_07.1.query.sqlpp
@@ -27,4 +27,4 @@
WHEN t IS UNKNOWN THEN (SELECT -3) // Should never enter this THEN branch.
ELSE 2.0/t
END
-FROM [0, 1, 2, 4, NULL, [0][-1]] t;
+FROM [0, 1, 2, 4, NULL, MISSING] t;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/constant_folding/constant_folding.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/constant_folding/constant_folding.2.query.sqlpp
new file mode 100644
index 0000000..83c3779
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/constant_folding/constant_folding.2.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: testing constant folding of OR with one TRUE argument
+ */
+
+select value true or x<=2
+from range(1, 4) x
+order by x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/constant_folding/constant_folding.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/constant_folding/constant_folding.3.query.sqlpp
new file mode 100644
index 0000000..85a77fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/constant_folding/constant_folding.3.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: testing constant folding of OR with one TRUE argument
+ */
+
+explain select value true or x<=2
+from range(1, 4) x
+order by x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/constant_folding/constant_folding.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/constant_folding/constant_folding.4.query.sqlpp
new file mode 100644
index 0000000..efb543f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/constant_folding/constant_folding.4.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: testing constant folding of OR with one FALSE argument
+ */
+
+select value false or x<=2
+from range(1, 4) x
+order by x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/constant_folding/constant_folding.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/constant_folding/constant_folding.5.query.sqlpp
new file mode 100644
index 0000000..4283fa6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/constant_folding/constant_folding.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: testing constant folding of OR with one FALSE argument
+ */
+
+explain select value false or x<=2
+from range(1, 4) x
+order by x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/constant_folding/constant_folding.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/constant_folding/constant_folding.6.query.sqlpp
new file mode 100644
index 0000000..b4b7c36
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/constant_folding/constant_folding.6.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: No constant folding of OR with non TRUE/FALSE argument
+ */
+
+explain select value null or x<=2
+from range(1, 4) x
+order by x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/constant_folding/constant_folding.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/constant_folding/constant_folding.7.query.sqlpp
new file mode 100644
index 0000000..517a996
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/constant_folding/constant_folding.7.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: No constant folding of OR with a non functional argument
+ */
+
+explain select value true or get_year(current_date()) < x
+from range(1, 4) x
+order by x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-2886/query-ASTERIXDB-2886.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-2886/query-ASTERIXDB-2886.1.ddl.sqlpp
new file mode 100644
index 0000000..ab6b723
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-2886/query-ASTERIXDB-2886.1.ddl.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test1 if exists;
+create dataverse test1;
+use test1;
+
+create dataset a(
+ c_a1 bigint not unknown,
+ c_a2 bigint,
+ c_a3 string
+) primary key c_a1;
+
+create index ia2 on a(c_a2);
+
+create index ia3 on a(c_a3);
+
+create dataset b(
+ c_b1 bigint not unknown,
+ c_b2 bigint,
+ c_b3 string
+) primary key c_b1;
+
+create index ib2 on b(c_b2);
+
+create index ib3 on b(c_b3);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-2886/query-ASTERIXDB-2886.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-2886/query-ASTERIXDB-2886.2.ddl.sqlpp
new file mode 100644
index 0000000..621d057
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-2886/query-ASTERIXDB-2886.2.ddl.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test2 if exists;
+create dataverse test2;
+use test2;
+
+create dataset a(
+ c_a1 bigint not unknown,
+ c_a2 bigint,
+ c_a3 string
+) primary key c_a1;
+
+create index ia2 on a(c_a2);
+
+create index ia3 on a(c_a3);
+
+create dataset b(
+ c_b1 bigint not unknown,
+ c_b2 bigint,
+ c_b3 string
+) primary key c_b1;
+
+create index ib2 on b(c_b2);
+
+create index ib3 on b(c_b3);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-2886/query-ASTERIXDB-2886.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-2886/query-ASTERIXDB-2886.3.query.sqlpp
new file mode 100644
index 0000000..e48a1e1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-2886/query-ASTERIXDB-2886.3.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+select ds.DataverseName, ds.DatasetName, dt.Derived.IsAnonymous, indexes
+from Metadata.`Dataset` as ds left join Metadata.`Datatype` dt
+on ds.DataverseName = dt.DataverseName and ds.DatatypeName = dt.DatatypeName
+let indexes = (
+ select idx.IndexName
+ from Metadata.`Index` as idx
+ where idx.DataverseName = ds.DataverseName and idx.DatasetName = ds.DatasetName
+ order by idx.IndexName
+)
+where ds.DataverseName like "test%"
+order by ds.DataverseName, ds.DatasetName;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/random/random.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/random/random.3.query.sqlpp
new file mode 100644
index 0000000..8b0aa23
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/random/random.3.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+FROM range(1, 3) AS r
+SELECT random(int8("12")) AS i8, random(int16("12")) AS i16, random(int32("12")) AS i32, random(int64("12")) AS i64,
+random(float("12")) AS float, random(double("12")) AS double;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/field-access/field-access.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/field-access/field-access.1.ddl.sqlpp
new file mode 100644
index 0000000..0d6ab35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/field-access/field-access.1.ddl.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+drop dataset samptable if exists;
+drop type samptabletype if exists;
+
+create type samptabletype as closed {
+ id: tinyint,
+ mf: tinyint?,
+ rf: { pid:tinyint}?
+};
+
+
+create dataset samptable(samptabletype) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/field-access/field-access.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/field-access/field-access.1.query.sqlpp
deleted file mode 100644
index 7be4593..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/field-access/field-access.1.query.sqlpp
+++ /dev/null
@@ -1,23 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-{
- 'null': {'a': null}.a,
- 'missing': {'b': missing}.b
-};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/field-access/field-access.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/field-access/field-access.2.update.sqlpp
new file mode 100644
index 0000000..8a7e88b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/field-access/field-access.2.update.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+insert into samptable ({'id' : 0, 'mf':missing, 'rf':{'pid':4}});
+insert into samptable ({'id' : 1, 'mf':null, 'rf':missing});
+insert into samptable ({'id' : 2, 'mf':5, 'rf':null});
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/field-access/field-access.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/field-access/field-access.3.query.sqlpp
new file mode 100644
index 0000000..bed3710
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/field-access/field-access.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+from samptable d
+select id, get_object_field_value(d, 'rf') as field_value,
+ get_object_field_value(d, 'rf') is missing as is_missing
+order by id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/field-access/field-access.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/field-access/field-access.4.query.sqlpp
new file mode 100644
index 0000000..553dc55
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/null-missing/field-access/field-access.4.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use test;
+
+from samptable d
+select id, get_object_field_value(d, 'mf') as field_value,
+ get_object_field_value(d, 'mf') is missing as is_missing
+order by id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/add_double/add_double.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/add_double/add_double.1.query.sqlpp
index 366f2b1..a5b1e1b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/add_double/add_double.1.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/numeric/add_double/add_double.1.query.sqlpp
@@ -18,4 +18,4 @@
*/
-{'result1':(double('-6.5d') + tinyint('+1')),'result2':(double('-6.5d') + smallint('2')),'result3':(double('-6.5d') + integer('+3')),'result4':(double('-6.5d') + bigint('-4')),'result5':(double('-6.5d') + float('-5.5f')),'result6':(double('-6.5d') + double('-6.5d')),'result7':(double('-6.5d') + null), 'result8':double('-6.5d') + [1.0][-1]};
+{'result1':(double('-6.5d') + tinyint('+1')),'result2':(double('-6.5d') + smallint('2')),'result3':(double('-6.5d') + integer('+3')),'result4':(double('-6.5d') + bigint('-4')),'result5':(double('-6.5d') + float('-5.5f')),'result6':(double('-6.5d') + double('-6.5d')),'result7':(double('-6.5d') + null), 'result8':double('-6.5d') + MISSING};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml
index 87bd204..a53a15e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml
@@ -57,11 +57,11 @@
<output-dir compare="Text">documentation-example</output-dir>
</compilation-unit>
</test-case>
- <!--test-case FilePath="objects/get-object-fields">
+ <test-case FilePath="objects/get-object-fields">
<compilation-unit name="tiny-social-example">
<output-dir compare="Text">tiny-social-example</output-dir>
</compilation-unit>
- </test-case!-->
+ </test-case>
<test-case FilePath="objects/get-object-fields">
<compilation-unit name="tiny-social-example-no-complex-types">
<output-dir compare="Text">tiny-social-example-no-complex-types</output-dir>
@@ -77,6 +77,11 @@
<output-dir compare="Text">tiny-social-example-only-records</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="objects/get-object-fields">
+ <compilation-unit name="missing-fields">
+ <output-dir compare="Text">missing-fields</output-dir>
+ </compilation-unit>
+ </test-case>
<test-case FilePath="objects/get-object-field-value">
<compilation-unit name="documentation-example">
<output-dir compare="Text">documentation-example</output-dir>
@@ -223,7 +228,7 @@
<expected-warn>Duplicate field name "name" (in line 22, at column 30)</expected-warn>
<expected-warn>Duplicate field name "id" (in line 22, at column 56)</expected-warn>
<expected-warn>Duplicate field name "f1" (in line 22, at column 70)</expected-warn>
- <expected-warn>Duplicate field name "id" (in line 22, at column 36)</expected-warn>
+ <expected-warn>Duplicate field name "id" (in line 22, at column 56)</expected-warn>
<expected-warn>Duplicate field name "f1" (in line 22, at column 83)</expected-warn>
<expected-warn>Duplicate field name "fname1" (in line 25, at column 45)</expected-warn>
</compilation-unit>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/get-object-fields/missing-fields/missing-fields.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/get-object-fields/missing-fields/missing-fields.1.ddl.sqlpp
new file mode 100644
index 0000000..4eeae30
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/get-object-fields/missing-fields/missing-fields.1.ddl.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Testing get-object-fields where some schema (complex) fields are missing or null
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+
+USE test;
+
+CREATE TYPE t1 AS {
+ inner_f: string
+};
+
+CREATE TYPE t2 AS {
+ id: int,
+ f: t1?
+};
+
+CREATE DATASET ds1(t2) PRIMARY KEY id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/get-object-fields/missing-fields/missing-fields.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/get-object-fields/missing-fields/missing-fields.2.update.sqlpp
new file mode 100644
index 0000000..93573bb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/get-object-fields/missing-fields/missing-fields.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;
+
+INSERT INTO ds1([
+{"id": 1, "f": null },
+{"id": 2},
+{"id": 3, "f": {"inner_f": "foo", "inner_f2": {"f3": "bar"} } }
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/get-object-fields/missing-fields/missing-fields.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/get-object-fields/missing-fields/missing-fields.3.query.sqlpp
new file mode 100644
index 0000000..819f46a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/get-object-fields/missing-fields/missing-fields.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE test;
+
+FROM ds1 AS t
+SELECT VALUE get_object_fields(t)
+ORDER BY t.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/get-object-fields/missing-fields/missing-fields.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/get-object-fields/missing-fields/missing-fields.4.query.sqlpp
new file mode 100644
index 0000000..d56e1e1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/get-object-fields/missing-fields/missing-fields.4.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.
+ */
+
+FROM `Metadata`.`Dataset` d, get_object_fields(d) f
+WHERE f.`is-open` = false
+SELECT DISTINCT VALUE f.`field-name`
+ORDER BY f.`field-name`;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-selection/btree-index-composite-key-04/btree-index-composite-key-04.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-selection/btree-index-composite-key-04/btree-index-composite-key-04.1.ddl.sqlpp
new file mode 100644
index 0000000..dea0b9c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-selection/btree-index-composite-key-04/btree-index-composite-key-04.1.ddl.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description : Index selection for non-enforced indexes.
+* Expected Res : Success
+*/
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create dataset TestOpen(c_id int64 not unknown) open type primary key c_id;
+
+create index idx_xyz on TestOpen(c_x:string, c_y:int64, c_z:string);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-selection/btree-index-composite-key-04/btree-index-composite-key-04.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-selection/btree-index-composite-key-04/btree-index-composite-key-04.2.update.sqlpp
new file mode 100644
index 0000000..2b7580d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-selection/btree-index-composite-key-04/btree-index-composite-key-04.2.update.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into TestOpen ({
+ "c_id": 1,
+ "c_x": "x1",
+ "c_y": 1,
+ "c_z": "z1",
+ "c_value": 1
+});
+insert into TestOpen ({
+ "c_id": 2,
+ "c_x": "x2",
+ "c_y": 2,
+ "c_z": "z2",
+ "c_value": 2
+});
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-selection/btree-index-composite-key-04/btree-index-composite-key-04.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-selection/btree-index-composite-key-04/btree-index-composite-key-04.3.query.sqlpp
new file mode 100644
index 0000000..9bd5957
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-non-enforced/index-selection/btree-index-composite-key-04/btree-index-composite-key-04.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select value t.c_value
+from TestOpen t
+where c_x = "x2" AND c_y > 1 AND c_z = "z2";
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets/all_datasets.10.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets/all_datasets.10.post.http
index 156c15e..ec40446 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets/all_datasets.10.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets/all_datasets.10.post.http
@@ -17,4 +17,4 @@
* under the License.
*/
-/admin/rebalance?nodes="asterix_nc1,asterix_nc2"
+/admin/rebalance?nodes=%22asterix_nc1%2Casterix_nc2%22
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets/all_datasets.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets/all_datasets.4.post.http
index 4e2b902..be4ae4b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets/all_datasets.4.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets/all_datasets.4.post.http
@@ -17,4 +17,4 @@
* under the License.
*/
-/admin/rebalance?nodes="asterix_nc1"
+/admin/rebalance?nodes=%22asterix_nc1%22
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.10.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.10.post.http
index 156c15e..ec40446 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.10.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.10.post.http
@@ -17,4 +17,4 @@
* under the License.
*/
-/admin/rebalance?nodes="asterix_nc1,asterix_nc2"
+/admin/rebalance?nodes=%22asterix_nc1%2Casterix_nc2%22
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.4.post.http
index 4e2b902..be4ae4b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.4.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.4.post.http
@@ -17,4 +17,4 @@
* under the License.
*/
-/admin/rebalance?nodes="asterix_nc1"
+/admin/rebalance?nodes=%22asterix_nc1%22
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/duplicate_location/duplicate_location.3.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/duplicate_location/duplicate_location.3.post.http
index f5b4573..48c45c1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/duplicate_location/duplicate_location.3.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/duplicate_location/duplicate_location.3.post.http
@@ -17,4 +17,4 @@
* under the License.
*/
-/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes="asterix_nc1,asterix_nc1,asterix_nc1"
+/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes=%22asterix_nc1%2Casterix_nc1%2Casterix_nc1%22
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/empty_location/empty_location.3.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/empty_location/empty_location.3.post.http
index e9e0e34..89aa420 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/empty_location/empty_location.3.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/empty_location/empty_location.3.post.http
@@ -17,4 +17,4 @@
* under the License.
*/
-/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes=""
+/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes=%22%22
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/identical_location/identical_location.3.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/identical_location/identical_location.3.post.http
index 33203f1..68d4a0f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/identical_location/identical_location.3.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/identical_location/identical_location.3.post.http
@@ -17,4 +17,4 @@
* under the License.
*/
-/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes="asterix_nc2,asterix_nc1"
+/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes=%22asterix_nc2%2Casterix_nc1%22&force=false
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/metadata/metadata.1.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/metadata/metadata.1.post.http
index e1b4921..84dc1b2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/metadata/metadata.1.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/metadata/metadata.1.post.http
@@ -17,4 +17,4 @@
* under the License.
*/
-/admin/rebalance?dataverseName=Metadata&datasetName=Dataset&nodes="asterix_nc1"
+/admin/rebalance?dataverseName=Metadata&datasetName=Dataset&nodes=%22asterix_nc1%22
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/miss_dataverse/miss_dataverse.3.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/miss_dataverse/miss_dataverse.3.post.http
index 4b49a72..f01dff9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/miss_dataverse/miss_dataverse.3.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/miss_dataverse/miss_dataverse.3.post.http
@@ -17,4 +17,4 @@
* under the License.
*/
-/admin/rebalance?datasetName=LineItem&nodes="asterix_nc1"
+/admin/rebalance?datasetName=LineItem&nodes=%22asterix_nc1%22
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/nonexist_dataset/nonexist_dataset.1.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/nonexist_dataset/nonexist_dataset.1.post.http
index 737c1fe..895232e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/nonexist_dataset/nonexist_dataset.1.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/nonexist_dataset/nonexist_dataset.1.post.http
@@ -17,4 +17,4 @@
* under the License.
*/
-/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes="asterix_nc1"
+/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes=%22asterix_nc1%22
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset/single_dataset.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset/single_dataset.4.post.http
index 737c1fe..895232e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset/single_dataset.4.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset/single_dataset.4.post.http
@@ -17,4 +17,4 @@
* under the License.
*/
-/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes="asterix_nc1"
+/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes=%22asterix_nc1%22
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset/single_dataset.8.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset/single_dataset.8.post.http
index 1a677fc..6399e8a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset/single_dataset.8.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset/single_dataset.8.post.http
@@ -17,4 +17,4 @@
* under the License.
*/
-/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes="asterix_nc1,asterix_nc2"
+/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes=%22asterix_nc1%2Casterix_nc2%22
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.4.post.http
index 737c1fe..895232e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.4.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.4.post.http
@@ -17,4 +17,4 @@
* under the License.
*/
-/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes="asterix_nc1"
+/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes=%22asterix_nc1%22
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.8.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.8.post.http
index 1a677fc..6399e8a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.8.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.8.post.http
@@ -17,4 +17,4 @@
* under the License.
*/
-/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes="asterix_nc1,asterix_nc2"
+/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes=%22asterix_nc1%2Casterix_nc2%22
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.4.post.http
index 737c1fe..895232e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.4.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.4.post.http
@@ -17,4 +17,4 @@
* under the License.
*/
-/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes="asterix_nc1"
+/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes=%22asterix_nc1%22
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.9.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.9.post.http
index 1a677fc..6399e8a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.9.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.9.post.http
@@ -17,4 +17,4 @@
* under the License.
*/
-/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes="asterix_nc1,asterix_nc2"
+/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes=%22asterix_nc1%2Casterix_nc2%22
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.4.post.http
index 737c1fe..895232e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.4.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.4.post.http
@@ -17,4 +17,4 @@
* under the License.
*/
-/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes="asterix_nc1"
+/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes=%22asterix_nc1%22
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.9.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.9.post.http
index 1a677fc..6399e8a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.9.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.9.post.http
@@ -17,4 +17,4 @@
* under the License.
*/
-/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes="asterix_nc1,asterix_nc2"
+/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes=%22asterix_nc1%2Casterix_nc2%22
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse/single_dataverse.10.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse/single_dataverse.10.post.http
index 5a1c111..2adb335 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse/single_dataverse.10.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse/single_dataverse.10.post.http
@@ -17,4 +17,4 @@
* under the License.
*/
-/admin/rebalance?dataverseName=tpch&nodes="asterix_nc1,asterix_nc2"
+/admin/rebalance?dataverseName=tpch&nodes=%22asterix_nc1%2Casterix_nc2%22
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse/single_dataverse.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse/single_dataverse.4.post.http
index d8f5812..f8e3b73 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse/single_dataverse.4.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse/single_dataverse.4.post.http
@@ -17,4 +17,4 @@
* under the License.
*/
-/admin/rebalance?dataverseName=tpch&nodes="asterix_nc1"
+/admin/rebalance?dataverseName=tpch&nodes=%22asterix_nc1%22
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.10.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.10.post.http
index 5a1c111..2adb335 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.10.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.10.post.http
@@ -17,4 +17,4 @@
* under the License.
*/
-/admin/rebalance?dataverseName=tpch&nodes="asterix_nc1,asterix_nc2"
+/admin/rebalance?dataverseName=tpch&nodes=%22asterix_nc1%2Casterix_nc2%22
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.4.post.http
index d8f5812..f8e3b73 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.4.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.4.post.http
@@ -17,4 +17,4 @@
* under the License.
*/
-/admin/rebalance?dataverseName=tpch&nodes="asterix_nc1"
+/admin/rebalance?dataverseName=tpch&nodes=%22asterix_nc1%22
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/index_01/index_01.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/index_01/index_01.1.ddl.sqlpp
new file mode 100644
index 0000000..7515b0b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/index_01/index_01.1.ddl.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description: Test that BTree index is used in query plan
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type test.Emp as
+ closed {
+ id : bigint,
+ fname : string,
+ lname : string,
+ age : bigint,
+ dept : string
+};
+
+create dataset employee(Emp) primary key id;
+
+create index idx_employee_fname on employee (fname) type btree;
+
+create index idx_employee_lname on employee (lname) type btree;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/index_01/index_01.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/index_01/index_01.2.update.sqlpp
new file mode 100644
index 0000000..eef17c1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/index_01/index_01.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.
+ */
+/*
+ * Description: Test that BTree index is used in query plan
+ */
+
+use test;
+
+load dataset employee using localfs ((`path`=`asterix_nc1://data/names.adm`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/index_01/index_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/index_01/index_01.3.query.sqlpp
new file mode 100644
index 0000000..aae6b95
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/index_01/index_01.3.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description: Test that BTree index is used in this query plan
+ */
+
+// requesttype=application/json
+
+// param $p_fname:json="Julio"
+// param $p_lname:json=null
+
+use test;
+
+select l.id, l.fname, l.lname, l.age
+from employee as l
+where
+ ($p_fname is null or l.fname = $p_fname )
+ and
+ ($p_lname is null or l.lname = $p_lname)
+order by l.id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/index_01/index_01.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/index_01/index_01.4.query.sqlpp
new file mode 100644
index 0000000..2b1c77c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/index_01/index_01.4.query.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description: Test that BTree index is used in this query plan
+ */
+
+// requesttype=application/json
+
+// param $p_fname:json="Julio"
+// param $p_lname:json="Isa"
+
+use test;
+
+select l.id, l.fname, l.lname, l.age
+from employee as l
+where
+ ($p_fname is null or l.fname = $p_fname )
+ and
+ ($p_lname is null or l.lname = $p_lname)
+order by l.id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr-ASTERIXDB-2949/substr-ASTERIXDB-2949.0.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr-ASTERIXDB-2949/substr-ASTERIXDB-2949.0.query.sqlpp
new file mode 100644
index 0000000..22105a4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/string/substr-ASTERIXDB-2949/substr-ASTERIXDB-2949.0.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Test the issue described in ASTERIXDB-2949
+ * Success : Yes
+ */
+
+
+SELECT SUBSTR("•\tABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789\n•\tabcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ\tABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789", 0, 1000) AS s;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-2815/query-ASTERIXDB-2815.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-2815/query-ASTERIXDB-2815.1.ddl.sqlpp
new file mode 100644
index 0000000..7de1e4e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-2815/query-ASTERIXDB-2815.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type t1 as {
+ _id: uuid
+};
+
+create dataset RawTweet(t1) primary key _id autogenerated;
+
+create dataset Evidence(t1) primary key _id autogenerated;
+
+create dataset Verification(t1) primary key _id autogenerated;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-2815/query-ASTERIXDB-2815.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-2815/query-ASTERIXDB-2815.2.update.sqlpp
new file mode 100644
index 0000000..29a4ff6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-2815/query-ASTERIXDB-2815.2.update.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into RawTweet ([
+ { "id" : 1, "full_text": "text_1" }, -- total evidence : 2
+ { "id" : 2, "full_text": "text_2" }, -- total evidence : 4 (*) -- satisfies the query
+ { "id" : 3, "full_text": "text_3" }, -- total evidence : 0
+ { "id" : 4, "full_text": "text_4" }, -- total evidence : 6 (*) -- satisfies the query
+ { "id" : 5, "full_text": "text_5" } -- total evidence : 1
+ ]);
+
+insert into Verification ([
+ { "ver_id" : 1001, "tweet_id": 1, "evidence": [ 2001, 2002 ] },
+ { "ver_id" : 1002, "tweet_id": 2, "evidence": [ 2003, 2004 ] },
+ { "ver_id" : 1003, "tweet_id": 2, "evidence": [ 2009, 2010 ] },
+ { "ver_id" : 1004, "tweet_id": 4, "evidence": [ 2004, 2005, 2006 ] },
+ { "ver_id" : 1005, "tweet_id": 4, "evidence": [ 2007, 2008, 2009 ] },
+ { "ver_id" : 1006, "tweet_id": 5, "evidence": [ 2001 ] }
+]);
+
+insert into Evidence ([
+ { "ev_id" : 2001, "url": "http://example.org/2001" },
+ { "ev_id" : 2002, "url": "http://example.org/2002" },
+ { "ev_id" : 2003, "url": "http://example.org/2003" },
+ { "ev_id" : 2004, "url": "http://example.org/2004" },
+ { "ev_id" : 2005, "url": "http://example.org/2005" },
+ { "ev_id" : 2006, "url": "http://example.org/2006" },
+ { "ev_id" : 2007, "url": "http://example.org/2007" },
+ { "ev_id" : 2008, "url": "http://example.org/2008" },
+ { "ev_id" : 2009, "url": "http://example.org/2009" },
+ { "ev_id" : 2010, "url": "http://example.org/2010" }
+]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-2815/query-ASTERIXDB-2815.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-2815/query-ASTERIXDB-2815.3.query.sqlpp
new file mode 100644
index 0000000..9072bb7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-2815/query-ASTERIXDB-2815.3.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+select t.id, array_sort(urls) urls
+from RawTweet t
+let urls = (
+ select distinct value e.url
+ from Verification v, v.evidence ve, Evidence e
+ where t.id = v.tweet_id and ve = e.ev_id
+)
+where array_count(urls) > 2
+order by t.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-2815/query-ASTERIXDB-2815.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-2815/query-ASTERIXDB-2815.4.query.sqlpp
new file mode 100644
index 0000000..d94a669
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-2815/query-ASTERIXDB-2815.4.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Test decorrelation of window functions
+ */
+
+use test;
+
+select t.id, array_sort(ranks) ranks
+from RawTweet t
+let ranks = (
+ select value rank() over(order by e.url)
+ from Verification v, v.evidence ve, Evidence e
+ where t.id = v.tweet_id and ve = e.ev_id
+)
+order by t.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-2815/query-ASTERIXDB-2815.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-2815/query-ASTERIXDB-2815.5.query.sqlpp
new file mode 100644
index 0000000..3bec7f9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-2815/query-ASTERIXDB-2815.5.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Test decorrelation of window functions
+ */
+
+use test;
+
+select t.id, array_sort(ranks) ranks
+from RawTweet t
+let ranks = (
+ select value rank() over(
+ partition by (tobigint(substring(e.url, -4)) % 2)
+ order by e.url
+ )
+ from Verification v, v.evidence ve, Evidence e
+ where t.id = v.tweet_id and ve = e.ev_id
+)
+order by t.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q19/q19.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q19/q19.2.update.sqlpp
index b59d644..fc0b44a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q19/q19.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q19/q19.2.update.sqlpp
@@ -20,14 +20,14 @@
use tpcds;
-load dataset customer_address using localfs ((`path`=`asterix_nc1://data/tpcds/customer_address.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset customer_address using localfs ((`path`=`asterix_nc1://data/tpcds/customer_address.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset item using localfs ((`path`=`asterix_nc1://data/tpcds/item.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset item using localfs ((`path`=`asterix_nc1://data/tpcds/item.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset customer using localfs ((`path`=`asterix_nc1://data/tpcds/customer.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset customer using localfs ((`path`=`asterix_nc1://data/tpcds/customer.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q25/q25.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q25/q25.2.update.sqlpp
index 3bf9f8d..efd670e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q25/q25.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q25/q25.2.update.sqlpp
@@ -20,14 +20,14 @@
use tpcds;
-load dataset item using localfs ((`path`=`asterix_nc1://data/tpcds/item.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset item using localfs ((`path`=`asterix_nc1://data/tpcds/item.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store_returns using localfs ((`path`=`asterix_nc1://data/tpcds/store_returns.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store_returns using localfs ((`path`=`asterix_nc1://data/tpcds/store_returns.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset catalog_sales using localfs ((`path`=`asterix_nc1://data/tpcds/catalog_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset catalog_sales using localfs ((`path`=`asterix_nc1://data/tpcds/catalog_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q29/q29.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q29/q29.2.update.sqlpp
index 3bf9f8d..efd670e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q29/q29.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q29/q29.2.update.sqlpp
@@ -20,14 +20,14 @@
use tpcds;
-load dataset item using localfs ((`path`=`asterix_nc1://data/tpcds/item.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset item using localfs ((`path`=`asterix_nc1://data/tpcds/item.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store_returns using localfs ((`path`=`asterix_nc1://data/tpcds/store_returns.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store_returns using localfs ((`path`=`asterix_nc1://data/tpcds/store_returns.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset catalog_sales using localfs ((`path`=`asterix_nc1://data/tpcds/catalog_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset catalog_sales using localfs ((`path`=`asterix_nc1://data/tpcds/catalog_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q30/q30.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q30/q30.2.update.sqlpp
index 190a142..8d5b408 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q30/q30.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q30/q30.2.update.sqlpp
@@ -20,10 +20,10 @@
use tpcds;
-load dataset customer_address using localfs ((`path`=`asterix_nc1://data/tpcds/customer_address.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset customer_address using localfs ((`path`=`asterix_nc1://data/tpcds/customer_address.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset web_returns using localfs ((`path`=`asterix_nc1://data/tpcds/web_returns.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset web_returns using localfs ((`path`=`asterix_nc1://data/tpcds/web_returns.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset customer using localfs ((`path`=`asterix_nc1://data/tpcds/customer.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset customer using localfs ((`path`=`asterix_nc1://data/tpcds/customer.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q34/q34.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q34/q34.2.update.sqlpp
index 97ea2e8..98accdd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q34/q34.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q34/q34.2.update.sqlpp
@@ -20,12 +20,12 @@
use tpcds;
-load dataset household_demographics using localfs ((`path`=`asterix_nc1://data/tpcds/household_demographics.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset household_demographics using localfs ((`path`=`asterix_nc1://data/tpcds/household_demographics.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset customer using localfs ((`path`=`asterix_nc1://data/tpcds/customer.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset customer using localfs ((`path`=`asterix_nc1://data/tpcds/customer.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q43/q43.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q43/q43.2.update.sqlpp
index bbf1838..654a1f1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q43/q43.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q43/q43.2.update.sqlpp
@@ -20,8 +20,8 @@
use tpcds;
-load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q46/q46.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q46/q46.2.update.sqlpp
index 6ab1d16..fc9d724 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q46/q46.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q46/q46.2.update.sqlpp
@@ -20,14 +20,14 @@
use tpcds;
-load dataset customer_address using localfs ((`path`=`asterix_nc1://data/tpcds/customer_address.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset customer_address using localfs ((`path`=`asterix_nc1://data/tpcds/customer_address.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset household_demographics using localfs ((`path`=`asterix_nc1://data/tpcds/household_demographics.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset household_demographics using localfs ((`path`=`asterix_nc1://data/tpcds/household_demographics.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset customer using localfs ((`path`=`asterix_nc1://data/tpcds/customer.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset customer using localfs ((`path`=`asterix_nc1://data/tpcds/customer.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q50/q50.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q50/q50.2.update.sqlpp
index d50d706..b870b65 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q50/q50.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q50/q50.2.update.sqlpp
@@ -20,10 +20,10 @@
use tpcds;
-load dataset store_returns using localfs ((`path`=`asterix_nc1://data/tpcds/store_returns.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store_returns using localfs ((`path`=`asterix_nc1://data/tpcds/store_returns.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q57/q57.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q57/q57.2.update.sqlpp
index 290b1ae..25b4d12 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q57/q57.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q57/q57.2.update.sqlpp
@@ -19,11 +19,11 @@
use tpcds;
-load dataset call_center using localfs ((`path`=`asterix_nc1://data/tpcds/call_center.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset call_center using localfs ((`path`=`asterix_nc1://data/tpcds/call_center.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset catalog_sales using localfs ((`path`=`asterix_nc1://data/tpcds/catalog_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset catalog_sales using localfs ((`path`=`asterix_nc1://data/tpcds/catalog_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset item using localfs ((`path`=`asterix_nc1://data/tpcds/item.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset item using localfs ((`path`=`asterix_nc1://data/tpcds/item.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q59/q59.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q59/q59.2.update.sqlpp
index 4cadf19..80bfe7a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q59/q59.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q59/q59.2.update.sqlpp
@@ -20,8 +20,8 @@
use tpcds;
-load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q62/q62.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q62/q62.2.update.sqlpp
index 2aab7e4..b1a240c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q62/q62.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q62/q62.2.update.sqlpp
@@ -20,12 +20,12 @@
use tpcds;
-load dataset web_sales using localfs ((`path`=`asterix_nc1://data/tpcds/web_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset web_sales using localfs ((`path`=`asterix_nc1://data/tpcds/web_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset web_site using localfs ((`path`=`asterix_nc1://data/tpcds/web_site.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset web_site using localfs ((`path`=`asterix_nc1://data/tpcds/web_site.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset warehouse using localfs ((`path`=`asterix_nc1://data/tpcds/warehouse.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset warehouse using localfs ((`path`=`asterix_nc1://data/tpcds/warehouse.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset ship_mode using localfs ((`path`=`asterix_nc1://data/tpcds/ship_mode.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset ship_mode using localfs ((`path`=`asterix_nc1://data/tpcds/ship_mode.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q63/q63.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q63/q63.2.update.sqlpp
index c2cef59..a56835d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q63/q63.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q63/q63.2.update.sqlpp
@@ -19,10 +19,10 @@
use tpcds;
-load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset item using localfs ((`path`=`asterix_nc1://data/tpcds/item.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset item using localfs ((`path`=`asterix_nc1://data/tpcds/item.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q73/q73.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q73/q73.2.update.sqlpp
index 97ea2e8..98accdd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q73/q73.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q73/q73.2.update.sqlpp
@@ -20,12 +20,12 @@
use tpcds;
-load dataset household_demographics using localfs ((`path`=`asterix_nc1://data/tpcds/household_demographics.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset household_demographics using localfs ((`path`=`asterix_nc1://data/tpcds/household_demographics.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset customer using localfs ((`path`=`asterix_nc1://data/tpcds/customer.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset customer using localfs ((`path`=`asterix_nc1://data/tpcds/customer.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q79/q79.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q79/q79.2.update.sqlpp
index 97ea2e8..98accdd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q79/q79.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q79/q79.2.update.sqlpp
@@ -20,12 +20,12 @@
use tpcds;
-load dataset household_demographics using localfs ((`path`=`asterix_nc1://data/tpcds/household_demographics.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset household_demographics using localfs ((`path`=`asterix_nc1://data/tpcds/household_demographics.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset customer using localfs ((`path`=`asterix_nc1://data/tpcds/customer.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset customer using localfs ((`path`=`asterix_nc1://data/tpcds/customer.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q81/q81.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q81/q81.2.update.sqlpp
index b7e0e29..9ce02ba 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q81/q81.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q81/q81.2.update.sqlpp
@@ -20,10 +20,10 @@
use tpcds;
-load dataset customer_address using localfs ((`path`=`asterix_nc1://data/tpcds/customer_address.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset customer_address using localfs ((`path`=`asterix_nc1://data/tpcds/customer_address.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset catalog_returns using localfs ((`path`=`asterix_nc1://data/tpcds/catalog_returns.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset catalog_returns using localfs ((`path`=`asterix_nc1://data/tpcds/catalog_returns.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset customer using localfs ((`path`=`asterix_nc1://data/tpcds/customer.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset customer using localfs ((`path`=`asterix_nc1://data/tpcds/customer.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q85/q85.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q85/q85.2.update.sqlpp
index fdb2d4f..821de94 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q85/q85.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q85/q85.2.update.sqlpp
@@ -20,16 +20,16 @@
use tpcds;
-load dataset web_page using localfs ((`path`=`asterix_nc1://data/tpcds/web_page.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset web_page using localfs ((`path`=`asterix_nc1://data/tpcds/web_page.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset customer_address using localfs ((`path`=`asterix_nc1://data/tpcds/customer_address.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset customer_address using localfs ((`path`=`asterix_nc1://data/tpcds/customer_address.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset reason using localfs ((`path`=`asterix_nc1://data/tpcds/reason.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset reason using localfs ((`path`=`asterix_nc1://data/tpcds/reason.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset customer_demographics using localfs ((`path`=`asterix_nc1://data/tpcds/customer_demographics.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset customer_demographics using localfs ((`path`=`asterix_nc1://data/tpcds/customer_demographics.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset web_sales using localfs ((`path`=`asterix_nc1://data/tpcds/web_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset web_sales using localfs ((`path`=`asterix_nc1://data/tpcds/web_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset web_returns using localfs ((`path`=`asterix_nc1://data/tpcds/web_returns.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset web_returns using localfs ((`path`=`asterix_nc1://data/tpcds/web_returns.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q88/q88.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q88/q88.2.update.sqlpp
index a673f5b..87cb458 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q88/q88.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q88/q88.2.update.sqlpp
@@ -20,10 +20,10 @@
use tpcds;
-load dataset household_demographics using localfs ((`path`=`asterix_nc1://data/tpcds/household_demographics.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset household_demographics using localfs ((`path`=`asterix_nc1://data/tpcds/household_demographics.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset time_dim using localfs ((`path`=`asterix_nc1://data/tpcds/time_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset time_dim using localfs ((`path`=`asterix_nc1://data/tpcds/time_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q89/q89.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q89/q89.2.update.sqlpp
index c2cef59..a56835d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q89/q89.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q89/q89.2.update.sqlpp
@@ -19,10 +19,10 @@
use tpcds;
-load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset item using localfs ((`path`=`asterix_nc1://data/tpcds/item.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset item using localfs ((`path`=`asterix_nc1://data/tpcds/item.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q90/q90.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q90/q90.2.update.sqlpp
index 319491a..512eaf2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q90/q90.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q90/q90.2.update.sqlpp
@@ -20,10 +20,10 @@
use tpcds;
-load dataset web_page using localfs ((`path`=`asterix_nc1://data/tpcds/web_page.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset web_page using localfs ((`path`=`asterix_nc1://data/tpcds/web_page.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset household_demographics using localfs ((`path`=`asterix_nc1://data/tpcds/household_demographics.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset household_demographics using localfs ((`path`=`asterix_nc1://data/tpcds/household_demographics.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset time_dim using localfs ((`path`=`asterix_nc1://data/tpcds/time_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset time_dim using localfs ((`path`=`asterix_nc1://data/tpcds/time_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset web_sales using localfs ((`path`=`asterix_nc1://data/tpcds/web_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset web_sales using localfs ((`path`=`asterix_nc1://data/tpcds/web_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q91/q91.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q91/q91.2.update.sqlpp
index 4599485..8d50a4a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q91/q91.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q91/q91.2.update.sqlpp
@@ -20,16 +20,16 @@
use tpcds;
-load dataset customer_address using localfs ((`path`=`asterix_nc1://data/tpcds/customer_address.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset customer_address using localfs ((`path`=`asterix_nc1://data/tpcds/customer_address.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset call_center using localfs ((`path`=`asterix_nc1://data/tpcds/call_center.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset call_center using localfs ((`path`=`asterix_nc1://data/tpcds/call_center.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset household_demographics using localfs ((`path`=`asterix_nc1://data/tpcds/household_demographics.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset household_demographics using localfs ((`path`=`asterix_nc1://data/tpcds/household_demographics.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset customer_demographics using localfs ((`path`=`asterix_nc1://data/tpcds/customer_demographics.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset customer_demographics using localfs ((`path`=`asterix_nc1://data/tpcds/customer_demographics.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset catalog_returns using localfs ((`path`=`asterix_nc1://data/tpcds/catalog_returns.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset catalog_returns using localfs ((`path`=`asterix_nc1://data/tpcds/catalog_returns.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset customer using localfs ((`path`=`asterix_nc1://data/tpcds/customer.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset customer using localfs ((`path`=`asterix_nc1://data/tpcds/customer.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q96/q96.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q96/q96.2.update.sqlpp
index a673f5b..87cb458 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q96/q96.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/q96/q96.2.update.sqlpp
@@ -20,10 +20,10 @@
use tpcds;
-load dataset household_demographics using localfs ((`path`=`asterix_nc1://data/tpcds/household_demographics.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset household_demographics using localfs ((`path`=`asterix_nc1://data/tpcds/household_demographics.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset time_dim using localfs ((`path`=`asterix_nc1://data/tpcds/time_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset time_dim using localfs ((`path`=`asterix_nc1://data/tpcds/time_dim.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
-load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`));
+load dataset store using localfs ((`path`=`asterix_nc1://data/tpcds/store.csv`),(`format`=`delimited-text`),(`delimiter`=`|`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/query-ASTERIXDB-1591/query-ASTERIXDB-1591.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/query-ASTERIXDB-1591/query-ASTERIXDB-1591.2.update.sqlpp
index 9401cee..194b60e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/query-ASTERIXDB-1591/query-ASTERIXDB-1591.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/query-ASTERIXDB-1591/query-ASTERIXDB-1591.2.update.sqlpp
@@ -20,22 +20,22 @@
use tpcds;
load dataset customer_address using localfs ((`path`=`asterix_nc1://data/tpcds/customer_address.csv`),
-(`format`=`delimited-text`), (`delimiter`=`|`));
+(`format`=`delimited-text`), (`delimiter`=`|`),(`null`=``));
load dataset customer using localfs ((`path`=`asterix_nc1://data/tpcds/customer.csv`),
-(`format`=`delimited-text`), (`delimiter`=`|`));
+(`format`=`delimited-text`), (`delimiter`=`|`),(`null`=``));
load dataset store_sales using localfs ((`path`=`asterix_nc1://data/tpcds/store_sales.csv`),
-(`format`=`delimited-text`), (`delimiter`=`|`));
+(`format`=`delimited-text`), (`delimiter`=`|`),(`null`=``));
load dataset date_dim using localfs ((`path`=`asterix_nc1://data/tpcds/date_dim.csv`),
-(`format`=`delimited-text`), (`delimiter`=`|`));
+(`format`=`delimited-text`), (`delimiter`=`|`),(`null`=``));
load dataset catalog_sales using localfs ((`path`=`asterix_nc1://data/tpcds/catalog_sales.csv`),
-(`format`=`delimited-text`), (`delimiter`=`|`));
+(`format`=`delimited-text`), (`delimiter`=`|`),(`null`=``));
load dataset catalog_returns using localfs ((`path`=`asterix_nc1://data/tpcds/catalog_returns.csv`),
-(`format`=`delimited-text`), (`delimiter`=`|`));
+(`format`=`delimited-text`), (`delimiter`=`|`),(`null`=``));
load dataset item using localfs ((`path`=`asterix_nc1://data/tpcds/item.csv`),
-(`format`=`delimited-text`), (`delimiter`=`|`));
+(`format`=`delimited-text`), (`delimiter`=`|`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/query-ASTERIXDB-1596/query-ASTERIXDB-1596.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/query-ASTERIXDB-1596/query-ASTERIXDB-1596.2.update.sqlpp
index 01cca8d..9c8aa41 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/query-ASTERIXDB-1596/query-ASTERIXDB-1596.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpcds/query-ASTERIXDB-1596/query-ASTERIXDB-1596.2.update.sqlpp
@@ -20,10 +20,10 @@
use tpcds;
load dataset catalog_sales using localfs ((`path`=`asterix_nc1://data/tpcds/catalog_sales.csv`),
-(`format`=`delimited-text`), (`delimiter`=`|`));
+(`format`=`delimited-text`), (`delimiter`=`|`),(`null`=``));
load dataset catalog_returns using localfs ((`path`=`asterix_nc1://data/tpcds/catalog_returns.csv`),
-(`format`=`delimited-text`), (`delimiter`=`|`));
+(`format`=`delimited-text`), (`delimiter`=`|`),(`null`=``));
load dataset item using localfs ((`path`=`asterix_nc1://data/tpcds/item.csv`),
-(`format`=`delimited-text`), (`delimiter`=`|`));
+(`format`=`delimited-text`), (`delimiter`=`|`),(`null`=``));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_nested/union_nested.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_nested/union_nested.1.ddl.sqlpp
new file mode 100644
index 0000000..d881c98
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_nested/union_nested.1.ddl.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Test various combinations of grouping sets
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type tenkType as closed {
+ unique1 : integer,
+ unique2 : integer,
+ two : integer,
+ four : integer,
+ ten : integer,
+ twenty : integer,
+ hundred : integer,
+ thousand : integer,
+ twothousand : integer,
+ fivethous : integer,
+ tenthous : integer,
+ odd100 : integer,
+ even100 : integer,
+ stringu1 : string,
+ stringu2 : string,
+ string4 : string
+};
+
+create dataset tenk(tenkType) primary key unique2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_nested/union_nested.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_nested/union_nested.2.update.sqlpp
new file mode 100644
index 0000000..7128175
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_nested/union_nested.2.update.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+load dataset tenk using localfs ((`path`=`asterix_nc1://data/tenk.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_nested/union_nested.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_nested/union_nested.3.query.sqlpp
new file mode 100644
index 0000000..ddd69bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/union/union_nested/union_nested.3.query.sqlpp
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 two, four, ten, twenty,
+ sum(unique1) as agg_sum
+from tenk
+group by two, four, ten, twenty
+
+union all
+select two, four, ten, twenty,
+ sum(unique1) as agg_sum
+from tenk
+group by -two as two, -four as four, -ten as ten, -twenty as twenty
+
+union all
+select two, four, ten, twenty,
+ sum(unique1) as agg_sum
+from tenk
+group by tostring(two) as two, tostring(four) as four, tostring(ten) as ten, tostring(twenty) as twenty
+
+union all
+select two, four, ten, null as twenty,
+ sum(unique1) as agg_sum
+from tenk
+group by two, four, ten
+
+union all
+select two, four, ten, null as twenty,
+ sum(unique1) as agg_sum
+from tenk
+group by -two as two, -four as four, -ten as ten
+
+union all
+select two, four, ten, null as twenty,
+ sum(unique1) as agg_sum
+from tenk
+group by tostring(two) as two, tostring(four) as four, tostring(ten) as ten
+
+union all
+select two, four, null as ten, null as twenty,
+ sum(unique1) as agg_sum
+from tenk
+group by two, four
+
+union all
+select two, four, null as ten, null as twenty,
+ sum(unique1) as agg_sum
+from tenk
+group by -two as two, -four as four
+
+union all
+select two, four, null as ten, null as twenty,
+ sum(unique1) as agg_sum
+from tenk
+group by tostring(two) as two, tostring(four) as four
+
+union all
+select two, null as four, null as ten, null as twenty,
+ sum(unique1) as agg_sum
+from tenk
+group by two
+
+union all
+select two, null as four, null as ten, null as twenty,
+ sum(unique1) as agg_sum
+from tenk
+group by -two as two
+
+union all
+select two, null as four, null as ten, null as twenty,
+ sum(unique1) as agg_sum
+from tenk
+group by tostring(two) as two
+
+order by two, four, ten, twenty;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/unnest/ASTERIXDB-2750_unnest_join/ASTERIXDB-2750_unnest_join.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/unnest/ASTERIXDB-2750_unnest_join/ASTERIXDB-2750_unnest_join.1.query.sqlpp
new file mode 100644
index 0000000..c608831
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/unnest/ASTERIXDB-2750_unnest_join/ASTERIXDB-2750_unnest_join.1.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+with
+ t1 as [ { "x": 1, "y": [10, 11, 12] }, { "x": 2, "y": [20, 21, 22] } ],
+ t2 as [ 100, 101, 102, 103 ]
+from t1 as t1, t1.y as y join t2 as t2 on true
+select value count(*)
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/unnest/ASTERIXDB-2750_unnest_join/ASTERIXDB-2750_unnest_join.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/unnest/ASTERIXDB-2750_unnest_join/ASTERIXDB-2750_unnest_join.2.query.sqlpp
new file mode 100644
index 0000000..0909785
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/unnest/ASTERIXDB-2750_unnest_join/ASTERIXDB-2750_unnest_join.2.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+with
+ t1 as [ { "x": 1, "y": [10, 11, 12] }, { "x": 2, "y": [20, 21, 22] } ],
+ t2 as [ 100, 101, 102, 103 ],
+ t3 as [ 1000, 1001, 1002, 1003, 1004 ]
+from t1 as t1, t1.y as y join t2 as t2 on true join t3 as t3 on true
+select value count(*)
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/unnest/ASTERIXDB-2844_unnest_syntax/ASTERIXDB-2844_unnest_syntax.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/unnest/ASTERIXDB-2844_unnest_syntax/ASTERIXDB-2844_unnest_syntax.1.query.sqlpp
new file mode 100644
index 0000000..110eefd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/unnest/ASTERIXDB-2844_unnest_syntax/ASTERIXDB-2844_unnest_syntax.1.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+with t as (select r, [r*10,r*10+1] ra from range(1, 2) r)
+select *
+from t unnest t.ra
+order by ra
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.10.query.sqlpp
new file mode 100644
index 0000000..57545a6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/window/misc_01/misc_01.10.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ * Description : Test multiple window functions in the same statement
+ * Expected Res : SUCCESS
+ */
+
+use test;
+
+select c2,
+ (
+ select nth_value(c2, 3)
+ over (partition by one order by c2 range between unbounded preceding and unbounded following)
+ as nth
+ from t1 x
+ ) as q1
+from t1 y
+order by c2;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/avg_mixed/avg_mixed.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/avg_mixed/avg_mixed.1.adm
index e3b97f5..e0ea36f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/avg_mixed/avg_mixed.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/avg_mixed/avg_mixed.1.adm
@@ -1 +1 @@
-[ ]
+6.0
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_double/serial_avg_double.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_double/serial_avg_double.1.adm
new file mode 100644
index 0000000..7e08e77
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_double/serial_avg_double.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "avg": 4.92 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_double_null/serial_avg_double_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_double_null/serial_avg_double_null.1.adm
new file mode 100644
index 0000000..877de08
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_double_null/serial_avg_double_null.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "avg": 150.0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate/avg_mixed/avg_mixed.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_empty/serial_sum_empty.1.adm
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate/avg_mixed/avg_mixed.1.ast
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_empty/serial_sum_empty.1.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_float/serial_avg_float.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_float/serial_avg_float.1.adm
new file mode 100644
index 0000000..7e08e77
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_float/serial_avg_float.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "avg": 4.92 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_float_null/serial_avg_float_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_float_null/serial_avg_float_null.1.adm
new file mode 100644
index 0000000..47ebaaa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_float_null/serial_avg_float_null.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "avg": 2.0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_int16/serial_avg_int16.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_int16/serial_avg_int16.1.adm
new file mode 100644
index 0000000..821af2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_int16/serial_avg_int16.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "avg": 4.6 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_int16_null/serial_avg_int16_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_int16_null/serial_avg_int16_null.1.adm
new file mode 100644
index 0000000..47ebaaa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_int16_null/serial_avg_int16_null.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "avg": 2.0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_int32/serial_avg_int32.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_int32/serial_avg_int32.1.adm
new file mode 100644
index 0000000..821af2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_int32/serial_avg_int32.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "avg": 4.6 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_int32_null/serial_avg_int32_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_int32_null/serial_avg_int32_null.1.adm
new file mode 100644
index 0000000..47ebaaa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_int32_null/serial_avg_int32_null.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "avg": 2.0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_int64/serial_avg_int64.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_int64/serial_avg_int64.1.adm
new file mode 100644
index 0000000..821af2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_int64/serial_avg_int64.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "avg": 4.6 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_int64_null/serial_avg_int64_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_int64_null/serial_avg_int64_null.1.adm
new file mode 100644
index 0000000..47ebaaa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_int64_null/serial_avg_int64_null.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "avg": 2.0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_int8/serial_avg_int8.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_int8/serial_avg_int8.1.adm
new file mode 100644
index 0000000..8e23c3e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_int8/serial_avg_int8.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "avg": 74.0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_int8_null/serial_avg_int8_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_int8_null/serial_avg_int8_null.1.adm
new file mode 100644
index 0000000..4c81aef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_int8_null/serial_avg_int8_null.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "avg": 100.0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_mixed/serial_avg_mixed.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_mixed/serial_avg_mixed.1.adm
new file mode 100644
index 0000000..9d306b7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-sql/serial_avg_mixed/serial_avg_mixed.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "avg": 2.5 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/avg_mixed/avg_mixed.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/avg_mixed/avg_mixed.1.adm
index e3b97f5..19765bd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/avg_mixed/avg_mixed.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/avg_mixed/avg_mixed.1.adm
@@ -1 +1 @@
-[ ]
+null
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_double/serial_avg_double.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_double/serial_avg_double.1.adm
new file mode 100644
index 0000000..7e08e77
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_double/serial_avg_double.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "avg": 4.92 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_double_null/serial_avg_double_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_double_null/serial_avg_double_null.1.adm
new file mode 100644
index 0000000..40a0b73
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_double_null/serial_avg_double_null.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "avg": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate/avg_mixed/avg_mixed.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_empty/serial_sum_empty.1.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate/avg_mixed/avg_mixed.1.ast
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_empty/serial_sum_empty.1.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_float/serial_avg_float.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_float/serial_avg_float.1.adm
new file mode 100644
index 0000000..7e08e77
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_float/serial_avg_float.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "avg": 4.92 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_float_null/serial_avg_float_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_float_null/serial_avg_float_null.1.adm
new file mode 100644
index 0000000..40a0b73
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_float_null/serial_avg_float_null.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "avg": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_int16/serial_avg_int16.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_int16/serial_avg_int16.1.adm
new file mode 100644
index 0000000..821af2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_int16/serial_avg_int16.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "avg": 4.6 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_int16_null/serial_avg_int16_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_int16_null/serial_avg_int16_null.1.adm
new file mode 100644
index 0000000..40a0b73
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_int16_null/serial_avg_int16_null.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "avg": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_int32/serial_avg_int32.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_int32/serial_avg_int32.1.adm
new file mode 100644
index 0000000..821af2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_int32/serial_avg_int32.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "avg": 4.6 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_int32_null/serial_avg_int32_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_int32_null/serial_avg_int32_null.1.adm
new file mode 100644
index 0000000..40a0b73
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_int32_null/serial_avg_int32_null.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "avg": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_int64/serial_avg_int64.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_int64/serial_avg_int64.1.adm
new file mode 100644
index 0000000..821af2f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_int64/serial_avg_int64.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "avg": 4.6 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_int64_null/serial_avg_int64_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_int64_null/serial_avg_int64_null.1.adm
new file mode 100644
index 0000000..40a0b73
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_int64_null/serial_avg_int64_null.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "avg": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_int8/serial_avg_int8.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_int8/serial_avg_int8.1.adm
new file mode 100644
index 0000000..8e23c3e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_int8/serial_avg_int8.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "avg": 74.0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_int8_null/serial_avg_int8_null.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_int8_null/serial_avg_int8_null.1.adm
new file mode 100644
index 0000000..40a0b73
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_int8_null/serial_avg_int8_null.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "avg": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_mixed/serial_avg_mixed.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_mixed/serial_avg_mixed.1.adm
new file mode 100644
index 0000000..40a0b73
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate/serial_avg_mixed/serial_avg_mixed.1.adm
@@ -0,0 +1 @@
+{ "gid": 1, "avg": null }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/get-query/get-query.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/get-query/get-query.1.adm
new file mode 100644
index 0000000..da1b8a2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/get-query/get-query.1.adm
@@ -0,0 +1 @@
+"Metadata"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/get-query/get-query.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/get-query/get-query.2.adm
new file mode 100644
index 0000000..da1b8a2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/get-query/get-query.2.adm
@@ -0,0 +1 @@
+"Metadata"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/get-query/get-query.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/get-query/get-query.3.adm
new file mode 100644
index 0000000..da1b8a2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/get-query/get-query.3.adm
@@ -0,0 +1 @@
+"Metadata"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/ignore-body-for-get/ignore-body-for-get.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/ignore-body-for-get/ignore-body-for-get.1.regexadm
new file mode 100644
index 0000000..73efe45
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/ignore-body-for-get/ignore-body-for-get.1.regexadm
@@ -0,0 +1 @@
+.*No statement provided.*
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/ignore-body-for-get/ignore-body-for-get.2.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/ignore-body-for-get/ignore-body-for-get.2.regexadm
new file mode 100644
index 0000000..73efe45
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/ignore-body-for-get/ignore-body-for-get.2.regexadm
@@ -0,0 +1 @@
+.*No statement provided.*
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/post-non-query/post-non-query.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/post-non-query/post-non-query.2.adm
new file mode 100644
index 0000000..4063cf1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/post-non-query/post-non-query.2.adm
@@ -0,0 +1 @@
+"success"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/csv-tsv-parser/csv-parser-001/csv-parser-001.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/csv-tsv-parser/csv-parser-001/csv-parser-001.2.adm
new file mode 100644
index 0000000..5c84fb8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/csv-tsv-parser/csv-parser-001/csv-parser-001.2.adm
@@ -0,0 +1,15 @@
+{ "f1": "a", "f2": "b", "f3": "c", "f4": "d", "f5": "e" }
+{ "f1": "0", "f2": ", boo", "f3": " 1", "f4": "2", "f5": "3" }
+{ "f1": "1", "f2": "", "f3": "", "f4": "❤", "f5": "" }
+{ "f1": "2", "f2": "3", "f3": "4", "f4": "\\n", "f5": "" }
+{ "f1": "3", "f2": "quoted \"f\" field", "f3": "", "f4": "", "f5": "" }
+{ "f1": "4", "f2": "4", "f3": "", "f4": "", "f5": "" }
+{ "f1": "5", "f2": "{\"vehicle\": \"car\", \"location\": [2.0, 0.1]}", "f3": "", "f4": "", "f5": "" }
+{ "f1": "6", "f2": "2", "f3": "3", "f4": "", "f5": "" }
+{ "f1": "7", "f2": "8", "f3": "9", "f4": "", "f5": "" }
+{ "f1": "8", "f2": "2", "f3": "3", "f4": "", "f5": "" }
+{ "f1": "9", "f2": "8", "f3": "9", "f4": "", "f5": "" }
+{ "f1": "10", "f2": "field\n\"f\"\nwith multiple lines", "f3": "", "f4": "", "f5": "" }
+{ "f1": "11", "f2": "4", "f3": "", "f4": "", "f5": "" }
+{ "f1": "12", "f2": "5", "f3": "ʤ", "f4": "", "f5": "" }
+{ "f1": "John", "f2": "Green", "f3": "111 downtown st.", "f4": "city, state", "f5": "99999" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/csv-tsv-parser/csv-parser-001/csv-parser-001.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/csv-tsv-parser/csv-parser-001/csv-parser-001.3.adm
new file mode 100644
index 0000000..80f5fb7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/csv-tsv-parser/csv-parser-001/csv-parser-001.3.adm
@@ -0,0 +1,32 @@
+{ "f1": "1", "f2": "?/ Text ending with a backslash / \\", "f3": "2000-09-03 07:12:22" }
+{ "f1": "2", "f2": "non quoted text!yes......", "f3": "2003-08-09 22:34:19" }
+{ "f1": "3", "f2": "Text with more sentences. Another sentence.", "f3": "2003-09-12 05:29:12" }
+{ "f1": "4", "f2": "Quoted text.. yes.", "f3": "2003-09-13 17:21:49" }
+{ "f1": "5", "f2": "Another text", "f3": "2003-01-21 23:31:41" }
+{ "f1": "6", "f2": "Text with' quotes.", "f3": "2003-09-14 20:15:50" }
+{ "f1": "7", "f2": "Text with quote's", "f3": "2003-09-14 18:34:03" }
+{ "f1": "8", "f2": "Text with quotes '", "f3": "2003-01-28 20:32:13" }
+{ "f1": "9", "f2": "Text with quotes \"", "f3": "2003-01-18 11:44:15" }
+{ "f1": "10", "f2": "Text with question marks!?!?", "f3": "2003-09-18 06:25:56" }
+{ "f1": "11", "f2": "\" Text that starts with quotes", "f3": "2003-09-12 00:31:24" }
+{ "f1": "12", "f2": "Text with \\\" backslash and quotes", "f3": "2003-09-13 20:30:06" }
+{ "f1": "13", "f2": "Text with \\\" backslash and quotes\\\"", "f3": "2003-09-14 16:20:36" }
+{ "f1": "14", "f2": "Text that has comma ,", "f3": "2003-09-12 08:21:18" }
+{ "f1": "15", "f2": "Text that has \",\" quoted comma", "f3": "2003-09-12 08:21:18" }
+{ "f1": "16", "f2": ",Text that has ", "f3": "2003-09-12 08:21:18" }
+{ "f1": "17", "f2": ",\",Text that has ", "f3": "2003-09-12 08:21:18" }
+{ "f1": "18", "f2": "Text with commas,inside it., yes", "f3": "2003-09-13 23:42:14" }
+{ "f1": "19", "f2": "Text that has \\n inside ", "f3": "2003-09-12 08:21:18" }
+{ "f1": "20", "f2": "Text that has \\\\\\n inside ", "f3": "2003-09-12 08:21:18" }
+{ "f1": "21", "f2": "text with :)", "f3": "2003-09-05 19:15:34" }
+{ "f1": "22", "f2": "Text that has \\\\\\\" inside \\\\", "f3": "2003-09-12 08:21:18" }
+{ "f1": "23", "f2": "Text that has \\\\\\\" inside \\\\\"", "f3": "2003-09-12 08:21:18" }
+{ "f1": "24", "f2": "\"text that spans multiple\nLines and more\nLines ane more and more\nLines ...\nAnd yet more lines\nAnd more\"", "f3": "2011-09-19 01:09:09" }
+{ "f1": "25", "f2": "Text \"\nmore lines", "f3": "2011-09-19 01:09:09" }
+{ "f1": "26", "f2": "\"\n", "f3": "2011-09-19 01:09:09" }
+{ "f1": "27", "f2": "Text", "f3": "" }
+{ "f1": "28", "f2": "Text", "f3": "2011-09-19 01:09:09" }
+{ "f1": "29", "f2": "Text\\.", "f3": "2011-09-19 01:09:09" }
+{ "f1": "30", "f2": "Text\\.", "f3": "2011-09-19 01:09:09" }
+{ "f1": "31", "f2": "\\.Text", "f3": "2011-09-19 01:09:09" }
+{ "f1": "32", "f2": "\\.Text", "f3": "2011-09-19 01:09:09" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/csv-tsv-parser/csv-parser-001/csv-parser-001.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/csv-tsv-parser/csv-parser-001/csv-parser-001.4.adm
new file mode 100644
index 0000000..5c61b4a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/csv-tsv-parser/csv-parser-001/csv-parser-001.4.adm
@@ -0,0 +1,4 @@
+{ "f1": "1", "f2": ",\", b", "f3": " 3", "f4": "4", "f5": "5" }
+{ "f1": ",\", b", "f2": "4", "f3": " 3", "f4": "4", "f5": "5" }
+{ "f1": "", "f2": "", "f3": "", "f4": "", "f5": "" }
+{ "f1": "dd", "f2": "", "f3": "", "f4": "", "f5": "" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/csv-tsv-parser/csv-parser-001/csv-parser-001.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/csv-tsv-parser/csv-parser-001/csv-parser-001.5.adm
new file mode 100644
index 0000000..4b80e26
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/csv-tsv-parser/csv-parser-001/csv-parser-001.5.adm
@@ -0,0 +1,13 @@
+{ "f1": 1, "f2": true, "f3": "text" }
+{ "f1": 2, "f2": false, "f3": "text" }
+{ "f1": 3, "f2": true, "f3": "text" }
+{ "f1": 4, "f2": true, "f3": null }
+{ "f1": 5, "f2": false, "f3": null }
+{ "f1": 6, "f2": true, "f3": "text\"\nmore lines" }
+{ "f1": 7, "f2": false, "f3": "\"\n" }
+{ "f1": 8, "f2": true, "f3": null }
+{ "f1": 9, "f2": false, "f3": "text\"" }
+{ "f1": 10, "f2": false, "f3": "text\\." }
+{ "f1": 11, "f2": true, "f3": "text\\." }
+{ "f1": null, "f2": false, "f3": "\\.text" }
+{ "f1": 13, "f2": true, "f3": "\\.text" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/csv-tsv-parser/csv-parser-001/csv-parser-001.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/csv-tsv-parser/csv-parser-001/csv-parser-001.6.adm
new file mode 100644
index 0000000..9a1d1c9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/csv-tsv-parser/csv-parser-001/csv-parser-001.6.adm
@@ -0,0 +1,3 @@
+{ "f1": "1", "f2": "", "f3": "good", "f4": "recommend" }
+{ "f1": "2", "f2": "", "f3": "bad", "f4": "not recommend" }
+{ "f1": "3", "f2": "", "f3": "good", "f4": "" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate/avg_mixed/avg_mixed.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results/csv-tsv-parser/csv-parser-001/csv-parser-001.7.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate/avg_mixed/avg_mixed.1.ast
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/csv-tsv-parser/csv-parser-001/csv-parser-001.7.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/csv-tsv-parser/tsv-parser-001/tsv-parser-001.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/csv-tsv-parser/tsv-parser-001/tsv-parser-001.2.adm
new file mode 100644
index 0000000..fbe287b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/csv-tsv-parser/tsv-parser-001/tsv-parser-001.2.adm
@@ -0,0 +1,28 @@
+{ "f1": 11, "f2": 55, "f3": "text field wih , charrrrrrrrrrr", "f4": true, "f5": 90, "f6": 0.666666667 }
+{ "f1": 12, "f2": 55, "f3": "text field with \" charrrrrrrrrr", "f4": false, "f5": 90, "f6": 0.666666667 }
+{ "f1": 14, "f2": 55, "f3": "text field with ' charrrrrrrrrr", "f4": false, "f5": 90, "f6": 0.666666667 }
+{ "f1": 15, "f2": 55, "f3": "text field with \\ charrrrrrrrrr", "f4": false, "f5": 90, "f6": 0.666666667 }
+{ "f1": 16, "f2": 55, "f3": "text field wih \\, char ", "f4": true, "f5": 90, "f6": 0.666666667 }
+{ "f1": 17, "f2": 55, "f3": "text field with \\\" charrrrrrrrr", "f4": false, "f5": 90, "f6": 0.666666667 }
+{ "f1": 18, "f2": 55, "f3": "text field with \\' charrrrrrrrr", "f4": false, "f5": 90, "f6": 0.666666667 }
+{ "f1": 19, "f2": 55, "f3": "text field with \\\\ charrrrrrrrr", "f4": false, "f5": 90, "f6": 0.666666667 }
+{ "f1": 20, "f2": 55, "f3": "text field ending with charr ,", "f4": false, "f5": 90, "f6": 0.666666667 }
+{ "f1": 21, "f2": 55, "f3": "text field ending with charr \"", "f4": false, "f5": 90, "f6": 0.666666667 }
+{ "f1": 22, "f2": 55, "f3": "text field ending with charr '", "f4": false, "f5": 90, "f6": 0.666666667 }
+{ "f1": 23, "f2": 55, "f3": "text field ending with charr \\", "f4": false, "f5": 90, "f6": 0.666666667 }
+{ "f1": 24, "f2": 55, "f3": "text field ending with charr \\,", "f4": false, "f5": 90, "f6": 0.666666667 }
+{ "f1": 25, "f2": 55, "f3": "text field ending with charr \\\"", "f4": false, "f5": 90, "f6": 0.666666667 }
+{ "f1": 26, "f2": 55, "f3": "text field ending with charr \\'", "f4": false, "f5": 90, "f6": 0.666666667 }
+{ "f1": 27, "f2": 55, "f3": "text field ending with charr \\\\", "f4": false, "f5": 90, "f6": 0.666666667 }
+{ "f1": 28, "f2": 55, "f3": ",text field starting with charr", "f4": false, "f5": 90, "f6": 0.666666667 }
+{ "f1": 29, "f2": 55, "f3": "\"text field starting with charr", "f4": false, "f5": 90, "f6": 0.666666667 }
+{ "f1": 30, "f2": 55, "f3": "'text field starting with charr", "f4": false, "f5": 90, "f6": 0.666666667 }
+{ "f1": 31, "f2": 55, "f3": "\\text field starting with charr", "f4": false, "f5": 90, "f6": 0.666666667 }
+{ "f1": 32, "f2": 55, "f3": "\\,text field starting with char", "f4": false, "f5": 90, "f6": 0.666666667 }
+{ "f1": 33, "f2": 55, "f3": "\\\"text field starting with char", "f4": false, "f5": 90, "f6": 0.666666667 }
+{ "f1": 34, "f2": 55, "f3": "\\'text field starting with char", "f4": false, "f5": 90, "f6": 0.666666667 }
+{ "f1": 35, "f2": 55, "f3": "\\\\text field starting with char", "f4": false, "f5": 90, "f6": 0.666666667 }
+{ "f1": 36, "f2": 55, "f3": "\"text field inside with char\"", "f4": false, "f5": 90, "f6": 0.666666667 }
+{ "f1": 37, "f2": 55, "f3": " text field with charrrrrrrrr ", "f4": false, "f5": 90, "f6": 0.666666667 }
+{ "f1": 38, "f2": 55, "f3": "text field with \"\" charrrrrrrrr", "f4": false, "f5": 90, "f6": 0.666666667 }
+{ "f1": 39, "f2": 55, "f3": "text field \"with\" charrrrrrrrrr", "f4": false, "f5": 90, "f6": 0.666666667 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/create-dataset-inline-type-1/create-dataset-inline-type-1.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/create-dataset-inline-type-1/create-dataset-inline-type-1.2.adm
new file mode 100644
index 0000000..992045b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/create-dataset-inline-type-1/create-dataset-inline-type-1.2.adm
@@ -0,0 +1,21 @@
+{ "en": "Dataset", "DatatypeName": "$d$t$i$A_Customers_Closed", "DatasetName": "A_Customers_Closed", "DatatypeDataverseName": "test" }
+{ "en": "Dataset", "DatatypeName": "$d$t$i$A_Customers_Default_Closed", "DatasetName": "A_Customers_Default_Closed", "DatatypeDataverseName": "test" }
+{ "en": "Dataset", "DatatypeName": "$d$t$i$A_Customers_Open", "DatasetName": "A_Customers_Open", "DatatypeDataverseName": "test" }
+{ "en": "Dataset", "DatatypeName": "$d$t$i$B_Orders_Closed", "DatasetName": "B_Orders_Closed", "DatatypeDataverseName": "test" }
+{ "en": "Dataset", "DatatypeName": "$d$t$i$B_Orders_Default_Closed", "DatasetName": "B_Orders_Default_Closed", "DatatypeDataverseName": "test" }
+{ "en": "Dataset", "DatatypeName": "$d$t$i$B_Orders_Open", "DatasetName": "B_Orders_Open", "DatatypeDataverseName": "test" }
+{ "en": "Dataset", "DatatypeName": "$d$t$i$C_Customers_Meta_Closed", "DatasetName": "C_Customers_Meta_Closed", "DatatypeDataverseName": "test", "MetatypeDataverseName": "test", "MetatypeName": "$d$t$m$C_Customers_Meta_Closed" }
+{ "en": "Dataset", "DatatypeName": "$d$t$i$C_Customers_Meta_Default_Closed", "DatasetName": "C_Customers_Meta_Default_Closed", "DatatypeDataverseName": "test", "MetatypeDataverseName": "test", "MetatypeName": "$d$t$m$C_Customers_Meta_Default_Closed" }
+{ "en": "Dataset", "DatatypeName": "$d$t$i$C_Customers_Meta_Open", "DatasetName": "C_Customers_Meta_Open", "DatatypeDataverseName": "test", "MetatypeDataverseName": "test", "MetatypeName": "$d$t$m$C_Customers_Meta_Open" }
+{ "en": "Datatype", "DatatypeName": "$d$t$i$A_Customers_Closed", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "c_custkey", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_name", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_phone", "FieldType": "string", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_comment", "FieldType": "string", "IsNullable": true, "IsMissable": true } ] } } }
+{ "en": "Datatype", "DatatypeName": "$d$t$i$A_Customers_Default_Closed", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "c_custkey", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_name", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_phone", "FieldType": "string", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_comment", "FieldType": "string", "IsNullable": true, "IsMissable": true } ] } } }
+{ "en": "Datatype", "DatatypeName": "$d$t$i$A_Customers_Open", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "c_custkey", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_name", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_phone", "FieldType": "string", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_comment", "FieldType": "string", "IsNullable": true, "IsMissable": true } ] } } }
+{ "en": "Datatype", "DatatypeName": "$d$t$i$B_Orders_Closed", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "o_orderkey", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "o_custkey", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "o_orderstatus", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "o_totalprice", "FieldType": "double", "IsNullable": false, "IsMissable": false }, { "FieldName": "o_orderdate", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "o_orderpriority", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "o_clerk", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "o_shippriority", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "o_comment", "FieldType": "string", "IsNullable": true, "IsMissable": true } ] } } }
+{ "en": "Datatype", "DatatypeName": "$d$t$i$B_Orders_Default_Closed", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "o_orderkey", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "o_custkey", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "o_orderstatus", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "o_totalprice", "FieldType": "double", "IsNullable": false, "IsMissable": false }, { "FieldName": "o_orderdate", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "o_orderpriority", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "o_clerk", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "o_shippriority", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "o_comment", "FieldType": "string", "IsNullable": true, "IsMissable": true } ] } } }
+{ "en": "Datatype", "DatatypeName": "$d$t$i$B_Orders_Open", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "o_orderkey", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "o_custkey", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "o_orderstatus", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "o_totalprice", "FieldType": "double", "IsNullable": false, "IsMissable": false }, { "FieldName": "o_orderdate", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "o_orderpriority", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "o_clerk", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "o_shippriority", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "o_comment", "FieldType": "string", "IsNullable": true, "IsMissable": true } ] } } }
+{ "en": "Datatype", "DatatypeName": "$d$t$i$C_Customers_Meta_Closed", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "c_custkey", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_name", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_phone", "FieldType": "string", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_comment", "FieldType": "string", "IsNullable": true, "IsMissable": true } ] } } }
+{ "en": "Datatype", "DatatypeName": "$d$t$i$C_Customers_Meta_Default_Closed", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "c_custkey", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_name", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_phone", "FieldType": "string", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_comment", "FieldType": "string", "IsNullable": true, "IsMissable": true } ] } } }
+{ "en": "Datatype", "DatatypeName": "$d$t$i$C_Customers_Meta_Open", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "c_custkey", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_name", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_phone", "FieldType": "string", "IsNullable": true, "IsMissable": true }, { "FieldName": "c_comment", "FieldType": "string", "IsNullable": true, "IsMissable": true } ] } } }
+{ "en": "Datatype", "DatatypeName": "$d$t$m$C_Customers_Meta_Closed", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "c_x", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_y", "FieldType": "int32", "IsNullable": true, "IsMissable": true } ] } } }
+{ "en": "Datatype", "DatatypeName": "$d$t$m$C_Customers_Meta_Default_Closed", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "c_x", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_y", "FieldType": "int32", "IsNullable": true, "IsMissable": true } ] } } }
+{ "en": "Datatype", "DatatypeName": "$d$t$m$C_Customers_Meta_Open", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "c_x", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "c_y", "FieldType": "int32", "IsNullable": true, "IsMissable": true } ] } } }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/create-dataset-inline-type-1/create-dataset-inline-type-1.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/create-dataset-inline-type-1/create-dataset-inline-type-1.4.adm
new file mode 100644
index 0000000..c227083
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/create-dataset-inline-type-1/create-dataset-inline-type-1.4.adm
@@ -0,0 +1 @@
+0
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/create-dataset-inline-type-2/create-dataset-inline-type-2.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/create-dataset-inline-type-2/create-dataset-inline-type-2.2.adm
new file mode 100644
index 0000000..3591912
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/create-dataset-inline-type-2/create-dataset-inline-type-2.2.adm
@@ -0,0 +1 @@
+{ "cnt": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/dataset-and-index-same-dataverse/dataset-and-index-same-dataverse.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/dataset-and-index-same-dataverse/dataset-and-index-same-dataverse.001.adm
new file mode 100644
index 0000000..057d80a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/ddl/dataset-and-index-same-dataverse/dataset-and-index-same-dataverse.001.adm
@@ -0,0 +1 @@
+{ "DataverseName": "test" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/create-drop-cltype/create-drop-cltype.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/create-drop-cltype/create-drop-cltype.1.adm
index e965793..d7f4edc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/create-drop-cltype/create-drop-cltype.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/create-drop-cltype/create-drop-cltype.1.adm
@@ -1 +1 @@
-{ "DataverseName": "test", "DatatypeName": "TestType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false }, { "FieldName": "salary", "FieldType": "double", "IsNullable": true }, { "FieldName": "name", "FieldType": "string", "IsNullable": false }, { "FieldName": "durtn", "FieldType": "duration", "IsNullable": true }, { "FieldName": "inter", "FieldType": "interval", "IsNullable": false }, { "FieldName": "dt", "FieldType": "date", "IsNullable": true }, { "FieldName": "tm", "FieldType": "time", "IsNullable": false }, { "FieldName": "pt", "FieldType": "point", "IsNullable": true } ] } }, "Timestamp": "Tue Jun 21 16:01:27 PDT 2016" }
+{ "DataverseName": "test", "DatatypeName": "TestType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": false, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "salary", "FieldType": "double", "IsNullable": true, "IsMissable": true }, { "FieldName": "name", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "durtn", "FieldType": "duration", "IsNullable": true, "IsMissable": true }, { "FieldName": "inter", "FieldType": "interval", "IsNullable": false, "IsMissable": false }, { "FieldName": "dt", "FieldType": "date", "IsNullable": true, "IsMissable": true }, { "FieldName": "tm", "FieldType": "time", "IsNullable": false, "IsMissable": false }, { "FieldName": "pt", "FieldType": "point", "IsNullable": true, "IsMissable": true } ] } }, "Timestamp": "Tue Jun 21 16:01:27 PDT 2016" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/create-drop-opntype/create-drop-opntype.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/create-drop-opntype/create-drop-opntype.1.adm
index e268379..e494b17 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/create-drop-opntype/create-drop-opntype.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/create-drop-opntype/create-drop-opntype.1.adm
@@ -1 +1 @@
-{ "DataverseName": "test", "DatatypeName": "TestType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false }, { "FieldName": "salary", "FieldType": "double", "IsNullable": true }, { "FieldName": "name", "FieldType": "string", "IsNullable": false }, { "FieldName": "durtn", "FieldType": "duration", "IsNullable": true }, { "FieldName": "inter", "FieldType": "interval", "IsNullable": false }, { "FieldName": "dt", "FieldType": "date", "IsNullable": true }, { "FieldName": "tm", "FieldType": "time", "IsNullable": false }, { "FieldName": "pt", "FieldType": "point", "IsNullable": true } ] } }, "Timestamp": "Tue Jun 21 16:01:27 PDT 2016" }
+{ "DataverseName": "test", "DatatypeName": "TestType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "salary", "FieldType": "double", "IsNullable": true, "IsMissable": true }, { "FieldName": "name", "FieldType": "string", "IsNullable": false, "IsMissable": false }, { "FieldName": "durtn", "FieldType": "duration", "IsNullable": true, "IsMissable": true }, { "FieldName": "inter", "FieldType": "interval", "IsNullable": false, "IsMissable": false }, { "FieldName": "dt", "FieldType": "date", "IsNullable": true, "IsMissable": true }, { "FieldName": "tm", "FieldType": "time", "IsNullable": false, "IsMissable": false }, { "FieldName": "pt", "FieldType": "point", "IsNullable": true, "IsMissable": true } ] } }, "Timestamp": "Tue Jun 21 16:01:27 PDT 2016" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/avg_mixed/avg_mixed.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-header/external_dataset.001.adm
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/avg_mixed/avg_mixed.1.ast
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-header/external_dataset.001.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/avg_mixed/avg_mixed.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-header/external_dataset.002.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/avg_mixed/avg_mixed.1.ast
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-header/external_dataset.002.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-header/external_dataset.003.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-header/external_dataset.003.adm
new file mode 100644
index 0000000..dc4d9c9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-header/external_dataset.003.adm
@@ -0,0 +1,2 @@
+{ "f1": 1, "f2": 2, "f3": 3, "f4": "str" }
+{ "f1": 1, "f2": 2, "f3": 3, "f4": "str" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-header/external_dataset.004.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-header/external_dataset.004.adm
new file mode 100644
index 0000000..1aae8c5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-header/external_dataset.004.adm
@@ -0,0 +1,6 @@
+{ "f1": 1, "f2": 2, "f3": 3, "f4": "str" }
+{ "f1": 1, "f2": 2, "f3": 3, "f4": "str" }
+{ "f1": 4, "f2": 5, "f3": 6, "f4": "rts" }
+{ "f1": 4, "f2": 5, "f3": 6, "f4": "rts" }
+{ "f1": 7, "f2": 8, "f3": 9, "f4": "srt" }
+{ "f1": 7, "f2": 8, "f3": 9, "f4": "srt" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/avg_mixed/avg_mixed.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-no-header/external_dataset.001.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/avg_mixed/avg_mixed.1.ast
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-no-header/external_dataset.001.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/avg_mixed/avg_mixed.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-no-header/external_dataset.002.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/avg_mixed/avg_mixed.1.ast
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-no-header/external_dataset.002.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-no-header/external_dataset.003.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-no-header/external_dataset.003.adm
new file mode 100644
index 0000000..dc4d9c9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-no-header/external_dataset.003.adm
@@ -0,0 +1,2 @@
+{ "f1": 1, "f2": 2, "f3": 3, "f4": "str" }
+{ "f1": 1, "f2": 2, "f3": 3, "f4": "str" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-no-header/external_dataset.004.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-no-header/external_dataset.004.adm
new file mode 100644
index 0000000..1aae8c5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-no-header/external_dataset.004.adm
@@ -0,0 +1,6 @@
+{ "f1": 1, "f2": 2, "f3": 3, "f4": "str" }
+{ "f1": 1, "f2": 2, "f3": 3, "f4": "str" }
+{ "f1": 4, "f2": 5, "f3": 6, "f4": "rts" }
+{ "f1": 4, "f2": 5, "f3": 6, "f4": "rts" }
+{ "f1": 7, "f2": 8, "f3": 9, "f4": "srt" }
+{ "f1": 7, "f2": 8, "f3": 9, "f4": "srt" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-warnings/external_dataset.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-warnings/external_dataset.001.adm
new file mode 100644
index 0000000..245b406
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-warnings/external_dataset.001.adm
@@ -0,0 +1,2 @@
+{ "f1": 1, "f2": 2, "f3": 3, "f4": "str" }
+{ "f1": 7, "f2": 8, "f3": 9, "f4": "srt" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-warnings/external_dataset.002.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-warnings/external_dataset.002.adm
new file mode 100644
index 0000000..7d3c940
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-warnings/external_dataset.002.adm
@@ -0,0 +1 @@
+{ "f1": 1, "f2": 2, "f3": 3, "f4": "str" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-warnings/external_dataset.003.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-warnings/external_dataset.003.adm
new file mode 100644
index 0000000..7d3c940
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-warnings/external_dataset.003.adm
@@ -0,0 +1 @@
+{ "f1": 1, "f2": 2, "f3": 3, "f4": "str" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-warnings/external_dataset.004.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-warnings/external_dataset.004.adm
new file mode 100644
index 0000000..334381b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-warnings/external_dataset.004.adm
@@ -0,0 +1,3 @@
+{ "f1": 1234, "f2": 12, "f3": 15.1, "f4": 16.1, "f5": "fine", "f6": "fine", "f7": true, "f8": false }
+{ "f1": 1234, "f2": 12, "f3": 15.1, "f4": 16.1, "f5": "fine", "f6": "fine", "f7": true, "f8": false }
+{ "f1": 1234, "f2": null, "f3": 15.1, "f4": null, "f5": "\\N", "f6": null, "f7": true, "f8": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-warnings/external_dataset.005.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-warnings/external_dataset.005.adm
new file mode 100644
index 0000000..334381b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-warnings/external_dataset.005.adm
@@ -0,0 +1,3 @@
+{ "f1": 1234, "f2": 12, "f3": 15.1, "f4": 16.1, "f5": "fine", "f6": "fine", "f7": true, "f8": false }
+{ "f1": 1234, "f2": 12, "f3": 15.1, "f4": 16.1, "f5": "fine", "f6": "fine", "f7": true, "f8": false }
+{ "f1": 1234, "f2": null, "f3": 15.1, "f4": null, "f5": "\\N", "f6": null, "f7": true, "f8": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-warnings/external_dataset.006.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-warnings/external_dataset.006.adm
new file mode 100644
index 0000000..c3ce0a1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-warnings/external_dataset.006.adm
@@ -0,0 +1 @@
+{ "f1": 1, "f2": "good", "f3": "recommend" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-warnings/external_dataset.007.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-warnings/external_dataset.007.adm
new file mode 100644
index 0000000..c56f0c5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv-warnings/external_dataset.007.adm
@@ -0,0 +1,2 @@
+{ "f1": 1, "f2": "good", "f3": "recommend" }
+{ "f1": 3, "f2": "good", "f3": "recommend" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv/csv/external_dataset.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv/csv/external_dataset.001.adm
new file mode 100644
index 0000000..4dbd63a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv/csv/external_dataset.001.adm
@@ -0,0 +1,30 @@
+{ "id": 1, "year": null, "review": "good", "details": "recommend" }
+{ "id": 1, "year": null, "review": "good", "details": "recommend" }
+{ "id": 1, "year": null, "review": "good", "details": "recommend" }
+{ "id": 1, "year": null, "review": "good", "details": "recommend" }
+{ "id": 1, "year": null, "review": "good", "details": "recommend" }
+{ "id": 2, "year": null, "review": "bad", "details": "not recommend" }
+{ "id": 2, "year": null, "review": "bad", "details": "not recommend" }
+{ "id": 2, "year": null, "review": "bad", "details": "not recommend" }
+{ "id": 2, "year": null, "review": "bad", "details": "not recommend" }
+{ "id": 2, "year": null, "review": "bad", "details": "not recommend" }
+{ "id": 3, "year": null, "review": "good", "details": null }
+{ "id": 3, "year": null, "review": "good", "details": null }
+{ "id": 3, "year": null, "review": "good", "details": null }
+{ "id": 3, "year": null, "review": "good", "details": null }
+{ "id": 3, "year": null, "review": "good", "details": null }
+{ "id": 4, "year": 2018, "review": "good", "details": "recommend" }
+{ "id": 4, "year": 2018, "review": "good", "details": "recommend" }
+{ "id": 4, "year": 2018, "review": "good", "details": "recommend" }
+{ "id": 4, "year": 2018, "review": "good", "details": "recommend" }
+{ "id": 4, "year": 2018, "review": "good", "details": "recommend" }
+{ "id": 5, "year": 2018, "review": "", "details": "not recommend" }
+{ "id": 5, "year": 2018, "review": "", "details": "not recommend" }
+{ "id": 5, "year": 2018, "review": "", "details": "not recommend" }
+{ "id": 5, "year": 2018, "review": "", "details": "not recommend" }
+{ "id": 5, "year": 2018, "review": "", "details": "not recommend" }
+{ "id": 6, "year": 2018, "review": "good", "details": null }
+{ "id": 6, "year": 2018, "review": "good", "details": null }
+{ "id": 6, "year": 2018, "review": "good", "details": null }
+{ "id": 6, "year": 2018, "review": "good", "details": null }
+{ "id": 6, "year": 2018, "review": "good", "details": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv/gz/external_dataset.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv/gz/external_dataset.001.adm
new file mode 100644
index 0000000..4dbd63a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv/gz/external_dataset.001.adm
@@ -0,0 +1,30 @@
+{ "id": 1, "year": null, "review": "good", "details": "recommend" }
+{ "id": 1, "year": null, "review": "good", "details": "recommend" }
+{ "id": 1, "year": null, "review": "good", "details": "recommend" }
+{ "id": 1, "year": null, "review": "good", "details": "recommend" }
+{ "id": 1, "year": null, "review": "good", "details": "recommend" }
+{ "id": 2, "year": null, "review": "bad", "details": "not recommend" }
+{ "id": 2, "year": null, "review": "bad", "details": "not recommend" }
+{ "id": 2, "year": null, "review": "bad", "details": "not recommend" }
+{ "id": 2, "year": null, "review": "bad", "details": "not recommend" }
+{ "id": 2, "year": null, "review": "bad", "details": "not recommend" }
+{ "id": 3, "year": null, "review": "good", "details": null }
+{ "id": 3, "year": null, "review": "good", "details": null }
+{ "id": 3, "year": null, "review": "good", "details": null }
+{ "id": 3, "year": null, "review": "good", "details": null }
+{ "id": 3, "year": null, "review": "good", "details": null }
+{ "id": 4, "year": 2018, "review": "good", "details": "recommend" }
+{ "id": 4, "year": 2018, "review": "good", "details": "recommend" }
+{ "id": 4, "year": 2018, "review": "good", "details": "recommend" }
+{ "id": 4, "year": 2018, "review": "good", "details": "recommend" }
+{ "id": 4, "year": 2018, "review": "good", "details": "recommend" }
+{ "id": 5, "year": 2018, "review": "", "details": "not recommend" }
+{ "id": 5, "year": 2018, "review": "", "details": "not recommend" }
+{ "id": 5, "year": 2018, "review": "", "details": "not recommend" }
+{ "id": 5, "year": 2018, "review": "", "details": "not recommend" }
+{ "id": 5, "year": 2018, "review": "", "details": "not recommend" }
+{ "id": 6, "year": 2018, "review": "good", "details": null }
+{ "id": 6, "year": 2018, "review": "good", "details": null }
+{ "id": 6, "year": 2018, "review": "good", "details": null }
+{ "id": 6, "year": 2018, "review": "good", "details": null }
+{ "id": 6, "year": 2018, "review": "good", "details": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv/mixed/external_dataset.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv/mixed/external_dataset.001.adm
new file mode 100644
index 0000000..7aec747
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/csv/mixed/external_dataset.001.adm
@@ -0,0 +1,60 @@
+{ "id": 1, "year": null, "review": "good", "details": "recommend" }
+{ "id": 1, "year": null, "review": "good", "details": "recommend" }
+{ "id": 1, "year": null, "review": "good", "details": "recommend" }
+{ "id": 1, "year": null, "review": "good", "details": "recommend" }
+{ "id": 1, "year": null, "review": "good", "details": "recommend" }
+{ "id": 1, "year": null, "review": "good", "details": "recommend" }
+{ "id": 1, "year": null, "review": "good", "details": "recommend" }
+{ "id": 1, "year": null, "review": "good", "details": "recommend" }
+{ "id": 1, "year": null, "review": "good", "details": "recommend" }
+{ "id": 1, "year": null, "review": "good", "details": "recommend" }
+{ "id": 2, "year": null, "review": "bad", "details": "not recommend" }
+{ "id": 2, "year": null, "review": "bad", "details": "not recommend" }
+{ "id": 2, "year": null, "review": "bad", "details": "not recommend" }
+{ "id": 2, "year": null, "review": "bad", "details": "not recommend" }
+{ "id": 2, "year": null, "review": "bad", "details": "not recommend" }
+{ "id": 2, "year": null, "review": "bad", "details": "not recommend" }
+{ "id": 2, "year": null, "review": "bad", "details": "not recommend" }
+{ "id": 2, "year": null, "review": "bad", "details": "not recommend" }
+{ "id": 2, "year": null, "review": "bad", "details": "not recommend" }
+{ "id": 2, "year": null, "review": "bad", "details": "not recommend" }
+{ "id": 3, "year": null, "review": "good", "details": null }
+{ "id": 3, "year": null, "review": "good", "details": null }
+{ "id": 3, "year": null, "review": "good", "details": null }
+{ "id": 3, "year": null, "review": "good", "details": null }
+{ "id": 3, "year": null, "review": "good", "details": null }
+{ "id": 3, "year": null, "review": "good", "details": null }
+{ "id": 3, "year": null, "review": "good", "details": null }
+{ "id": 3, "year": null, "review": "good", "details": null }
+{ "id": 3, "year": null, "review": "good", "details": null }
+{ "id": 3, "year": null, "review": "good", "details": null }
+{ "id": 4, "year": 2018, "review": "good", "details": "recommend" }
+{ "id": 4, "year": 2018, "review": "good", "details": "recommend" }
+{ "id": 4, "year": 2018, "review": "good", "details": "recommend" }
+{ "id": 4, "year": 2018, "review": "good", "details": "recommend" }
+{ "id": 4, "year": 2018, "review": "good", "details": "recommend" }
+{ "id": 4, "year": 2018, "review": "good", "details": "recommend" }
+{ "id": 4, "year": 2018, "review": "good", "details": "recommend" }
+{ "id": 4, "year": 2018, "review": "good", "details": "recommend" }
+{ "id": 4, "year": 2018, "review": "good", "details": "recommend" }
+{ "id": 4, "year": 2018, "review": "good", "details": "recommend" }
+{ "id": 5, "year": 2018, "review": "", "details": "not recommend" }
+{ "id": 5, "year": 2018, "review": "", "details": "not recommend" }
+{ "id": 5, "year": 2018, "review": "", "details": "not recommend" }
+{ "id": 5, "year": 2018, "review": "", "details": "not recommend" }
+{ "id": 5, "year": 2018, "review": "", "details": "not recommend" }
+{ "id": 5, "year": 2018, "review": "", "details": "not recommend" }
+{ "id": 5, "year": 2018, "review": "", "details": "not recommend" }
+{ "id": 5, "year": 2018, "review": "", "details": "not recommend" }
+{ "id": 5, "year": 2018, "review": "", "details": "not recommend" }
+{ "id": 5, "year": 2018, "review": "", "details": "not recommend" }
+{ "id": 6, "year": 2018, "review": "good", "details": null }
+{ "id": 6, "year": 2018, "review": "good", "details": null }
+{ "id": 6, "year": 2018, "review": "good", "details": null }
+{ "id": 6, "year": 2018, "review": "good", "details": null }
+{ "id": 6, "year": 2018, "review": "good", "details": null }
+{ "id": 6, "year": 2018, "review": "good", "details": null }
+{ "id": 6, "year": 2018, "review": "good", "details": null }
+{ "id": 6, "year": 2018, "review": "good", "details": null }
+{ "id": 6, "year": 2018, "review": "good", "details": null }
+{ "id": 6, "year": 2018, "review": "good", "details": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/custom-buffer-size/external_dataset.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/custom-buffer-size/external_dataset.001.adm
new file mode 100644
index 0000000..187a8cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/custom-buffer-size/external_dataset.001.adm
@@ -0,0 +1 @@
+{ "count": 100 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/custom-buffer-size/external_dataset.002.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/custom-buffer-size/external_dataset.002.adm
new file mode 100644
index 0000000..187a8cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/custom-buffer-size/external_dataset.002.adm
@@ -0,0 +1 @@
+{ "count": 100 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/custom-buffer-size/external_dataset.003.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/custom-buffer-size/external_dataset.003.adm
new file mode 100644
index 0000000..5db606c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/custom-buffer-size/external_dataset.003.adm
@@ -0,0 +1 @@
+{ "count": 25 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/custom-buffer-size/external_dataset.004.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/custom-buffer-size/external_dataset.004.adm
new file mode 100644
index 0000000..7660e7e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/custom-buffer-size/external_dataset.004.adm
@@ -0,0 +1,25 @@
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/custom-buffer-size/external_dataset.005.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/custom-buffer-size/external_dataset.005.adm
new file mode 100644
index 0000000..5db606c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/custom-buffer-size/external_dataset.005.adm
@@ -0,0 +1 @@
+{ "count": 25 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/custom-buffer-size/external_dataset.006.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/custom-buffer-size/external_dataset.006.adm
new file mode 100644
index 0000000..7643986
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/custom-buffer-size/external_dataset.006.adm
@@ -0,0 +1,25 @@
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/definition-does-not-exist/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/definition-does-not-exist/result.001.adm
new file mode 100644
index 0000000..c1a0ea2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/definition-does-not-exist/result.001.adm
@@ -0,0 +1 @@
+{ "count": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/empty-string-definition/external_dataset.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/empty-string-definition/external_dataset.001.adm
new file mode 100644
index 0000000..187a8cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/empty-string-definition/external_dataset.001.adm
@@ -0,0 +1 @@
+{ "count": 100 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/exclude-1/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/exclude-1/result.001.adm
new file mode 100644
index 0000000..207f123
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/exclude-1/result.001.adm
@@ -0,0 +1 @@
+{ "count": 12 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/exclude-2/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/exclude-2/result.001.adm
new file mode 100644
index 0000000..95204aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/exclude-2/result.001.adm
@@ -0,0 +1 @@
+{ "count": 6 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/exclude-3/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/exclude-3/result.001.adm
new file mode 100644
index 0000000..09eace4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/exclude-3/result.001.adm
@@ -0,0 +1 @@
+{ "count": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/exclude-4/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/exclude-4/result.001.adm
new file mode 100644
index 0000000..95204aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/exclude-4/result.001.adm
@@ -0,0 +1 @@
+{ "count": 6 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/exclude-5/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/exclude-5/result.001.adm
new file mode 100644
index 0000000..8ac9199
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/exclude-5/result.001.adm
@@ -0,0 +1 @@
+{ "count": 11 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/exclude-6/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/exclude-6/result.001.adm
new file mode 100644
index 0000000..207f123
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/exclude-6/result.001.adm
@@ -0,0 +1 @@
+{ "count": 12 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/exclude-all/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/exclude-all/result.001.adm
new file mode 100644
index 0000000..c1a0ea2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/exclude-all/result.001.adm
@@ -0,0 +1 @@
+{ "count": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-1/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-1/result.001.adm
new file mode 100644
index 0000000..df462fe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-1/result.001.adm
@@ -0,0 +1 @@
+{ "count": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-10/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-10/result.001.adm
new file mode 100644
index 0000000..c1a0ea2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-10/result.001.adm
@@ -0,0 +1 @@
+{ "count": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-11/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-11/result.001.adm
new file mode 100644
index 0000000..95204aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-11/result.001.adm
@@ -0,0 +1 @@
+{ "count": 6 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-12/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-12/result.001.adm
new file mode 100644
index 0000000..c1a0ea2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-12/result.001.adm
@@ -0,0 +1 @@
+{ "count": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-2/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-2/result.001.adm
new file mode 100644
index 0000000..09eace4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-2/result.001.adm
@@ -0,0 +1 @@
+{ "count": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-2/result.002.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-2/result.002.adm
new file mode 100644
index 0000000..9b5f94a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-2/result.002.adm
@@ -0,0 +1,4 @@
+{ "id": 1 }
+{ "id": 2 }
+{ "id": 4 }
+{ "id": 5 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-3/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-3/result.001.adm
new file mode 100644
index 0000000..09eace4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-3/result.001.adm
@@ -0,0 +1 @@
+{ "count": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-4/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-4/result.001.adm
new file mode 100644
index 0000000..09eace4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-4/result.001.adm
@@ -0,0 +1 @@
+{ "count": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-5/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-5/result.001.adm
new file mode 100644
index 0000000..8637a38
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-5/result.001.adm
@@ -0,0 +1 @@
+{ "count": 3 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-6/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-6/result.001.adm
new file mode 100644
index 0000000..95204aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-6/result.001.adm
@@ -0,0 +1 @@
+{ "count": 6 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-7/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-7/result.001.adm
new file mode 100644
index 0000000..95204aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-7/result.001.adm
@@ -0,0 +1 @@
+{ "count": 6 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-8/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-8/result.001.adm
new file mode 100644
index 0000000..95204aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-8/result.001.adm
@@ -0,0 +1 @@
+{ "count": 6 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-9/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-9/result.001.adm
new file mode 100644
index 0000000..95204aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/include-exclude/include-9/result.001.adm
@@ -0,0 +1 @@
+{ "count": 6 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json-warnings/json-warnings.003.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json-warnings/json-warnings.003.adm
new file mode 100644
index 0000000..5bae5d1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json-warnings/json-warnings.003.adm
@@ -0,0 +1 @@
+{ "double_value": 150.11 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/gz/external_dataset.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/gz/external_dataset.001.adm
new file mode 100644
index 0000000..187a8cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/gz/external_dataset.001.adm
@@ -0,0 +1 @@
+{ "count": 100 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/gz/external_dataset.002.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/gz/external_dataset.002.adm
new file mode 100644
index 0000000..187a8cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/gz/external_dataset.002.adm
@@ -0,0 +1 @@
+{ "count": 100 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/gz/external_dataset.003.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/gz/external_dataset.003.adm
new file mode 100644
index 0000000..5db606c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/gz/external_dataset.003.adm
@@ -0,0 +1 @@
+{ "count": 25 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/gz/external_dataset.004.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/gz/external_dataset.004.adm
new file mode 100644
index 0000000..7660e7e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/gz/external_dataset.004.adm
@@ -0,0 +1,25 @@
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/gz/external_dataset.005.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/gz/external_dataset.005.adm
new file mode 100644
index 0000000..5db606c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/gz/external_dataset.005.adm
@@ -0,0 +1 @@
+{ "count": 25 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/gz/external_dataset.006.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/gz/external_dataset.006.adm
new file mode 100644
index 0000000..7643986
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/gz/external_dataset.006.adm
@@ -0,0 +1,25 @@
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/json/external_dataset.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/json/external_dataset.001.adm
new file mode 100644
index 0000000..187a8cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/json/external_dataset.001.adm
@@ -0,0 +1 @@
+{ "count": 100 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/json/external_dataset.002.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/json/external_dataset.002.adm
new file mode 100644
index 0000000..187a8cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/json/external_dataset.002.adm
@@ -0,0 +1 @@
+{ "count": 100 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/json/external_dataset.003.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/json/external_dataset.003.adm
new file mode 100644
index 0000000..5db606c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/json/external_dataset.003.adm
@@ -0,0 +1 @@
+{ "count": 25 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/json/external_dataset.004.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/json/external_dataset.004.adm
new file mode 100644
index 0000000..7660e7e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/json/external_dataset.004.adm
@@ -0,0 +1,25 @@
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/json/external_dataset.005.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/json/external_dataset.005.adm
new file mode 100644
index 0000000..5db606c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/json/external_dataset.005.adm
@@ -0,0 +1 @@
+{ "count": 25 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/json/external_dataset.006.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/json/external_dataset.006.adm
new file mode 100644
index 0000000..7643986
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/json/external_dataset.006.adm
@@ -0,0 +1,25 @@
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/json/external_dataset.007.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/json/external_dataset.007.adm
new file mode 100644
index 0000000..86babba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/json/external_dataset.007.adm
@@ -0,0 +1 @@
+50128
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/mixed/external_dataset.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/mixed/external_dataset.001.adm
new file mode 100644
index 0000000..8e8fe53
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/mixed/external_dataset.001.adm
@@ -0,0 +1 @@
+{ "count": 200 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/mixed/external_dataset.002.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/mixed/external_dataset.002.adm
new file mode 100644
index 0000000..8e8fe53
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/mixed/external_dataset.002.adm
@@ -0,0 +1 @@
+{ "count": 200 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/mixed/external_dataset.003.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/mixed/external_dataset.003.adm
new file mode 100644
index 0000000..6540472
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/mixed/external_dataset.003.adm
@@ -0,0 +1 @@
+{ "count": 50 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/mixed/external_dataset.004.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/mixed/external_dataset.004.adm
new file mode 100644
index 0000000..9a52dea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/mixed/external_dataset.004.adm
@@ -0,0 +1,50 @@
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/mixed/external_dataset.005.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/mixed/external_dataset.005.adm
new file mode 100644
index 0000000..6540472
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/mixed/external_dataset.005.adm
@@ -0,0 +1 @@
+{ "count": 50 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/mixed/external_dataset.006.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/mixed/external_dataset.006.adm
new file mode 100644
index 0000000..722dfe1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/json/mixed/external_dataset.006.adm
@@ -0,0 +1,50 @@
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 1, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 2, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/jsonl/external_dataset.002.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/jsonl/external_dataset.002.adm
new file mode 100644
index 0000000..bf2b0e1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/jsonl/external_dataset.002.adm
@@ -0,0 +1,50 @@
+{ "id": 1, "year": null, "quarter": null, "review": "good" }
+{ "id": 2, "year": null, "quarter": null, "review": "good" }
+{ "id": 3, "year": 2018, "quarter": null, "review": "good" }
+{ "id": 4, "year": 2018, "quarter": null, "review": "bad" }
+{ "id": 5, "year": 2018, "quarter": 1, "review": "good" }
+{ "id": 6, "year": 2018, "quarter": 1, "review": "bad" }
+{ "id": 7, "year": 2018, "quarter": 2, "review": "good" }
+{ "id": 8, "year": 2018, "quarter": 2, "review": "bad" }
+{ "id": 9, "year": 2019, "quarter": null, "review": "good" }
+{ "id": 10, "year": 2019, "quarter": null, "review": "bad" }
+{ "id": 11, "year": 2019, "quarter": 1, "review": "good" }
+{ "id": 12, "year": 2019, "quarter": 1, "review": "bad" }
+{ "id": 13, "year": 2019, "quarter": 2, "review": "good" }
+{ "id": 14, "year": 2019, "quarter": 2, "review": "bad" }
+{ "id": 15, "year": 2019, "quarter": null, "review": "good" }
+{ "id": 16, "year": 2019, "quarter": null, "review": "bad" }
+{ "id": 17, "year": 2019, "quarter": 1, "review": "good" }
+{ "id": 18, "year": 2019, "quarter": 1, "review": "bad" }
+{ "id": 19, "year": 2019, "quarter": 2, "review": "good" }
+{ "id": 20, "year": 2019, "quarter": 2, "review": "bad" }
+{ "id": 21, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 22, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ] }
+{ "id": 23, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 24, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3 ] }
+{ "id": 25, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3 ] }
+{ "id": 26, "year": null, "quarter": null, "review": "good", "array": [ 1, 2, 3 ], "nested": { "id": 1 } }
+{ "id": 27, "year": null, "quarter": null, "review": "good", "array": [ 1, [ 1, 2 ], [ 1 ] ], "nested": { "id": 1 }, "nested2": [ { "id": 1 } ] }
+{ "id": 28, "year": 2018, "quarter": null, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ { "nested": { "array": [ 1, 2 ] } } ] } } ] }
+{ "id": 29, "year": 2018, "quarter": null, "review": "bad", "array": [ 1, 2, 3, { "nested1": { "id": 1, "nested2": { "id": 2, "nested3": [ { "nested4": null } ] } } } ] }
+{ "id": 30, "year": 2018, "quarter": 1, "review": "good", "array": [ 1, 2, 3, { "nested": { "array": [ 1, 2 ] } } ] }
+{ "id": 31, "year": null, "quarter": null, "review": "good" }
+{ "id": 32, "year": null, "quarter": null, "review": "good" }
+{ "id": 33, "year": 2018, "quarter": null, "review": "good" }
+{ "id": 34, "year": 2018, "quarter": null, "review": "bad" }
+{ "id": 35, "year": 2018, "quarter": 1, "review": "good" }
+{ "id": 36, "year": 2018, "quarter": 1, "review": "bad" }
+{ "id": 37, "year": 2018, "quarter": 2, "review": "good" }
+{ "id": 38, "year": 2018, "quarter": 2, "review": "bad" }
+{ "id": 39, "year": 2019, "quarter": null, "review": "good" }
+{ "id": 40, "year": 2019, "quarter": null, "review": "bad" }
+{ "id": 41, "year": 2019, "quarter": 1, "review": "good" }
+{ "id": 42, "year": 2019, "quarter": 1, "review": "bad" }
+{ "id": 43, "year": 2019, "quarter": 2, "review": "good" }
+{ "id": 44, "year": 2019, "quarter": 2, "review": "bad" }
+{ "id": 45, "year": 2019, "quarter": null, "review": "good" }
+{ "id": 46, "year": 2019, "quarter": null, "review": "bad" }
+{ "id": 47, "year": 2019, "quarter": 1, "review": "good" }
+{ "id": 48, "year": 2019, "quarter": 1, "review": "bad" }
+{ "id": 49, "year": 2019, "quarter": 2, "review": "good" }
+{ "id": 50, "year": 2019, "quarter": 2, "review": "bad" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/no-files-returned/definition-points-to-nothing/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/no-files-returned/definition-points-to-nothing/result.001.adm
new file mode 100644
index 0000000..c1a0ea2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/no-files-returned/definition-points-to-nothing/result.001.adm
@@ -0,0 +1 @@
+{ "count": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/no-files-returned/exclude-all-files/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/no-files-returned/exclude-all-files/result.001.adm
new file mode 100644
index 0000000..c1a0ea2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/no-files-returned/exclude-all-files/result.001.adm
@@ -0,0 +1 @@
+{ "count": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/no-files-returned/include-no-files/result.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/no-files-returned/include-no-files/result.001.adm
new file mode 100644
index 0000000..c1a0ea2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/no-files-returned/include-no-files/result.001.adm
@@ -0,0 +1 @@
+{ "count": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/over-1000-objects/over-1000-objects.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/over-1000-objects/over-1000-objects.001.adm
new file mode 100644
index 0000000..b610b1d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/over-1000-objects/over-1000-objects.001.adm
@@ -0,0 +1 @@
+{ "count": 2999 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/avg_mixed/avg_mixed.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv-header/external_dataset.001.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/avg_mixed/avg_mixed.1.ast
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv-header/external_dataset.001.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/avg_mixed/avg_mixed.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv-header/external_dataset.002.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/avg_mixed/avg_mixed.1.ast
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv-header/external_dataset.002.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv-header/external_dataset.003.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv-header/external_dataset.003.adm
new file mode 100644
index 0000000..eb83800
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv-header/external_dataset.003.adm
@@ -0,0 +1,2 @@
+{ "f1": 1, "f2": 2, "f3": 3, "f4": "\"str\"" }
+{ "f1": 1, "f2": 2, "f3": 3, "f4": "\"str\"" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv-header/external_dataset.004.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv-header/external_dataset.004.adm
new file mode 100644
index 0000000..8d60432
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv-header/external_dataset.004.adm
@@ -0,0 +1,6 @@
+{ "f1": 1, "f2": 2, "f3": 3, "f4": "\"str\"" }
+{ "f1": 1, "f2": 2, "f3": 3, "f4": "\"str\"" }
+{ "f1": 4, "f2": 5, "f3": 6, "f4": "\"rts\"" }
+{ "f1": 4, "f2": 5, "f3": 6, "f4": "\"rts\"" }
+{ "f1": 7, "f2": 8, "f3": 9, "f4": "\"srt\"" }
+{ "f1": 7, "f2": 8, "f3": 9, "f4": "\"srt\"" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/avg_mixed/avg_mixed.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv-no-header/external_dataset.001.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/avg_mixed/avg_mixed.1.ast
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv-no-header/external_dataset.001.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/avg_mixed/avg_mixed.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv-no-header/external_dataset.002.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/avg_mixed/avg_mixed.1.ast
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv-no-header/external_dataset.002.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv-no-header/external_dataset.003.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv-no-header/external_dataset.003.adm
new file mode 100644
index 0000000..eb83800
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv-no-header/external_dataset.003.adm
@@ -0,0 +1,2 @@
+{ "f1": 1, "f2": 2, "f3": 3, "f4": "\"str\"" }
+{ "f1": 1, "f2": 2, "f3": 3, "f4": "\"str\"" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv-no-header/external_dataset.004.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv-no-header/external_dataset.004.adm
new file mode 100644
index 0000000..8d60432
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv-no-header/external_dataset.004.adm
@@ -0,0 +1,6 @@
+{ "f1": 1, "f2": 2, "f3": 3, "f4": "\"str\"" }
+{ "f1": 1, "f2": 2, "f3": 3, "f4": "\"str\"" }
+{ "f1": 4, "f2": 5, "f3": 6, "f4": "\"rts\"" }
+{ "f1": 4, "f2": 5, "f3": 6, "f4": "\"rts\"" }
+{ "f1": 7, "f2": 8, "f3": 9, "f4": "\"srt\"" }
+{ "f1": 7, "f2": 8, "f3": 9, "f4": "\"srt\"" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv-warnings/external_dataset.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv-warnings/external_dataset.001.adm
new file mode 100644
index 0000000..ac2354d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv-warnings/external_dataset.001.adm
@@ -0,0 +1,2 @@
+{ "f1": 1, "f2": 2, "f3": 3, "f4": "\"str\"" }
+{ "f1": 7, "f2": 8, "f3": 9, "f4": "\"srt\"" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv/gz/external_dataset.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv/gz/external_dataset.001.adm
new file mode 100644
index 0000000..e04a6b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv/gz/external_dataset.001.adm
@@ -0,0 +1,30 @@
+{ "id": 1, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 1, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 1, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 1, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 1, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 2, "year": null, "review": "\"bad\"", "details": "\"not recommend\"" }
+{ "id": 2, "year": null, "review": "\"bad\"", "details": "\"not recommend\"" }
+{ "id": 2, "year": null, "review": "\"bad\"", "details": "\"not recommend\"" }
+{ "id": 2, "year": null, "review": "\"bad\"", "details": "\"not recommend\"" }
+{ "id": 2, "year": null, "review": "\"bad\"", "details": "\"not recommend\"" }
+{ "id": 3, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 3, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 3, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 3, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 3, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 4, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 4, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 4, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 4, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 4, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 5, "year": 2018, "review": "", "details": "\"not recommend\"" }
+{ "id": 5, "year": 2018, "review": "", "details": "\"not recommend\"" }
+{ "id": 5, "year": 2018, "review": "", "details": "\"not recommend\"" }
+{ "id": 5, "year": 2018, "review": "", "details": "\"not recommend\"" }
+{ "id": 5, "year": 2018, "review": "", "details": "\"not recommend\"" }
+{ "id": 6, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 6, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 6, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 6, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 6, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv/mixed/external_dataset.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv/mixed/external_dataset.001.adm
new file mode 100644
index 0000000..4823286
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv/mixed/external_dataset.001.adm
@@ -0,0 +1,60 @@
+{ "id": 1, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 1, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 1, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 1, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 1, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 1, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 1, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 1, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 1, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 1, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 2, "year": null, "review": "\"bad\"", "details": "\"not recommend\"" }
+{ "id": 2, "year": null, "review": "\"bad\"", "details": "\"not recommend\"" }
+{ "id": 2, "year": null, "review": "\"bad\"", "details": "\"not recommend\"" }
+{ "id": 2, "year": null, "review": "\"bad\"", "details": "\"not recommend\"" }
+{ "id": 2, "year": null, "review": "\"bad\"", "details": "\"not recommend\"" }
+{ "id": 2, "year": null, "review": "\"bad\"", "details": "\"not recommend\"" }
+{ "id": 2, "year": null, "review": "\"bad\"", "details": "\"not recommend\"" }
+{ "id": 2, "year": null, "review": "\"bad\"", "details": "\"not recommend\"" }
+{ "id": 2, "year": null, "review": "\"bad\"", "details": "\"not recommend\"" }
+{ "id": 2, "year": null, "review": "\"bad\"", "details": "\"not recommend\"" }
+{ "id": 3, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 3, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 3, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 3, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 3, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 3, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 3, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 3, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 3, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 3, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 4, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 4, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 4, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 4, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 4, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 4, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 4, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 4, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 4, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 4, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 5, "year": 2018, "review": "", "details": "\"not recommend\"" }
+{ "id": 5, "year": 2018, "review": "", "details": "\"not recommend\"" }
+{ "id": 5, "year": 2018, "review": "", "details": "\"not recommend\"" }
+{ "id": 5, "year": 2018, "review": "", "details": "\"not recommend\"" }
+{ "id": 5, "year": 2018, "review": "", "details": "\"not recommend\"" }
+{ "id": 5, "year": 2018, "review": "", "details": "\"not recommend\"" }
+{ "id": 5, "year": 2018, "review": "", "details": "\"not recommend\"" }
+{ "id": 5, "year": 2018, "review": "", "details": "\"not recommend\"" }
+{ "id": 5, "year": 2018, "review": "", "details": "\"not recommend\"" }
+{ "id": 5, "year": 2018, "review": "", "details": "\"not recommend\"" }
+{ "id": 6, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 6, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 6, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 6, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 6, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 6, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 6, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 6, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 6, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 6, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv/tsv/external_dataset.001.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv/tsv/external_dataset.001.adm
new file mode 100644
index 0000000..e04a6b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-dataset/aws/s3/tsv/tsv/external_dataset.001.adm
@@ -0,0 +1,30 @@
+{ "id": 1, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 1, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 1, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 1, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 1, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 2, "year": null, "review": "\"bad\"", "details": "\"not recommend\"" }
+{ "id": 2, "year": null, "review": "\"bad\"", "details": "\"not recommend\"" }
+{ "id": 2, "year": null, "review": "\"bad\"", "details": "\"not recommend\"" }
+{ "id": 2, "year": null, "review": "\"bad\"", "details": "\"not recommend\"" }
+{ "id": 2, "year": null, "review": "\"bad\"", "details": "\"not recommend\"" }
+{ "id": 3, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 3, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 3, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 3, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 3, "year": null, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 4, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 4, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 4, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 4, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 4, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 5, "year": 2018, "review": "", "details": "\"not recommend\"" }
+{ "id": 5, "year": 2018, "review": "", "details": "\"not recommend\"" }
+{ "id": 5, "year": 2018, "review": "", "details": "\"not recommend\"" }
+{ "id": 5, "year": 2018, "review": "", "details": "\"not recommend\"" }
+{ "id": 5, "year": 2018, "review": "", "details": "\"not recommend\"" }
+{ "id": 6, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 6, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 6, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 6, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
+{ "id": 6, "year": 2018, "review": "\"good\"", "details": "\"recommend\"" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/feeds_01/feeds_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/feeds_01/feeds_01.1.adm
index 85cd967..8ada33c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/feeds_01/feeds_01.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/feeds/feeds_01/feeds_01.1.adm
@@ -1 +1 @@
-{ "DataverseName": "feeds", "FeedName": "TweetFeed", "AdapterConfiguration": {{ { "Name": "path", "Value": "asterix_nc1://data/twitter/obamatweets.adm" }, { "Name": "feed", "Value": "TweetFeed" }, { "Name": "adapter-name", "Value": "localfs" }, { "Name": "is-feed", "Value": "true" }, { "Name": "parser", "Value": "adm" }, { "Name": "reader", "Value": "localfs" }, { "Name": "format", "Value": "adm" }, { "Name": "tuple-interval", "Value": "10" }, { "Name": "type-name", "Value": "TweetType" }, { "Name": "dataverse", "Value": "feeds" } }}, "Timestamp": "Thu Dec 07 19:22:41 PST 2017" }
+{ "DataverseName": "feeds", "FeedName": "TweetFeed", "AdapterConfiguration": {{ { "Name": "dataset-dataverse", "Value": "feeds" }, { "Name": "path", "Value": "asterix_nc1://data/twitter/obamatweets.adm" }, { "Name": "feed", "Value": "TweetFeed" }, { "Name": "adapter-name", "Value": "localfs" }, { "Name": "is-feed", "Value": "true" }, { "Name": "parser", "Value": "adm" }, { "Name": "reader", "Value": "localfs" }, { "Name": "format", "Value": "adm" }, { "Name": "tuple-interval", "Value": "10" }, { "Name": "type-name", "Value": "TweetType" } }}, "Timestamp": "Tue Mar 31 10:30:06 PDT 2020" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/group-by/gby-case-01/gby-case-01.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/group-by/gby-case-01/gby-case-01.3.adm
new file mode 100644
index 0000000..11331e8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/group-by/gby-case-01/gby-case-01.3.adm
@@ -0,0 +1,2 @@
+{ "x": 1, "res": 6.0 }
+{ "x": 2, "res": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/group-by/gby-case-01/gby-case-01.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/group-by/gby-case-01/gby-case-01.4.adm
new file mode 100644
index 0000000..11331e8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/group-by/gby-case-01/gby-case-01.4.adm
@@ -0,0 +1,2 @@
+{ "x": 1, "res": 6.0 }
+{ "x": 2, "res": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/group-by/gby-cross-join/gby-cross-join.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/group-by/gby-cross-join/gby-cross-join.3.adm
new file mode 100644
index 0000000..e1e3cf1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/group-by/gby-cross-join/gby-cross-join.3.adm
@@ -0,0 +1,2 @@
+{ "cnt": 6, "x": "a" }
+{ "cnt": 6, "x": "b" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/group-by/gby-nested-01/gby-nested-01.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/group-by/gby-nested-01/gby-nested-01.3.adm
new file mode 100644
index 0000000..caba454
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/group-by/gby-nested-01/gby-nested-01.3.adm
@@ -0,0 +1,2 @@
+{ "p_id": 1, "ct": [ { "ct_sum": 92, "ct_avg": 46.0 }, { "ct_id": "LAS", "ct_sum": 16, "ct_avg": 8.0 }, { "ct_id": "RNO", "ct_sum": 14, "ct_avg": 7.0 }, { "ct_id": "SFO", "ct_sum": 10, "ct_avg": 5.0 }, { "ct_id": "SJC", "ct_sum": 12, "ct_avg": 6.0 } ], "st": [ { "st_id": "CA", "st_sum": 64, "st_avg": 16.0 }, { "st_id": "NV", "st_sum": 80, "st_avg": 20.0 } ], "p_name": "PRODUCT_1" }
+{ "p_id": 2, "ct": [ ], "st": [ ], "p_name": "PRODUCT_2" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/empty-dataset/empty-dataset.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/empty-dataset/empty-dataset.03.adm
new file mode 100644
index 0000000..2cddb05
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/empty-dataset/empty-dataset.03.adm
@@ -0,0 +1 @@
+{ "ds1": { "id": "1", "f": 3 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate/avg_mixed/avg_mixed.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/empty-dataset/empty-dataset.04.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate/avg_mixed/avg_mixed.1.ast
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/empty-dataset/empty-dataset.04.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate/avg_mixed/avg_mixed.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/empty-dataset/empty-dataset.05.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate/avg_mixed/avg_mixed.1.ast
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/empty-dataset/empty-dataset.05.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate/avg_mixed/avg_mixed.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/empty-dataset/empty-dataset.06.adm
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate/avg_mixed/avg_mixed.1.ast
copy to asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/empty-dataset/empty-dataset.06.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/loj-02-push-select/loj-02-push-select.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/loj-02-push-select/loj-02-push-select.03.adm
new file mode 100644
index 0000000..bd1fb36
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/loj-02-push-select/loj-02-push-select.03.adm
@@ -0,0 +1,2 @@
+{ "t1_left": { "id": 3, "x": 3, "a": 13 } }
+{ "t1_left": { "id": 4, "x": 4, "a": 14 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/loj-02-push-select/loj-02-push-select.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/loj-02-push-select/loj-02-push-select.04.adm
new file mode 100644
index 0000000..bd1fb36
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/loj-02-push-select/loj-02-push-select.04.adm
@@ -0,0 +1,2 @@
+{ "t1_left": { "id": 3, "x": 3, "a": 13 } }
+{ "t1_left": { "id": 4, "x": 4, "a": 14 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/loj-02-push-select/loj-02-push-select.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/loj-02-push-select/loj-02-push-select.05.adm
new file mode 100644
index 0000000..bd1fb36
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/loj-02-push-select/loj-02-push-select.05.adm
@@ -0,0 +1,2 @@
+{ "t1_left": { "id": 3, "x": 3, "a": 13 } }
+{ "t1_left": { "id": 4, "x": 4, "a": 14 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/loj-02-push-select/loj-02-push-select.06.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/loj-02-push-select/loj-02-push-select.06.adm
new file mode 100644
index 0000000..e9d3c37
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/loj-02-push-select/loj-02-push-select.06.adm
@@ -0,0 +1,2 @@
+{ "t1_left": { "id": 1, "x": 1, "a": 11 }, "t2_right": { "id": 1, "y": 1, "b": 111 } }
+{ "t1_left": { "id": 2, "x": 2, "a": 12 }, "t2_right": { "id": 2, "y": 2, "b": 112 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/loj-03-no-listify/loj-03-no-listify.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/loj-03-no-listify/loj-03-no-listify.3.adm
new file mode 100644
index 0000000..466b846
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/loj-03-no-listify/loj-03-no-listify.3.adm
@@ -0,0 +1,2 @@
+{ "cnt_all": 6, "cnt_x": 2, "cnt_y": 2, "cnt_z": 2, "taskId": 1 }
+{ "cnt_all": 6, "cnt_x": 2, "cnt_y": 2, "cnt_z": 2, "taskId": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/query-ASTERIXDB-2857/query-ASTERIXDB-2857.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/query-ASTERIXDB-2857/query-ASTERIXDB-2857.3.adm
new file mode 100644
index 0000000..1a31db8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/query-ASTERIXDB-2857/query-ASTERIXDB-2857.3.adm
@@ -0,0 +1,2 @@
+{ "t0_unique1": 1891, "t1_unique1": 1891, "t2_unique1": 1891 }
+{ "t0_unique1": 8800, "t1_unique1": 8800, "t2_unique1": 8800 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/right_branch_opt_1/right_branch_opt_1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/right_branch_opt_1/right_branch_opt_1.1.adm
new file mode 100644
index 0000000..19eb6b6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/leftouterjoin/right_branch_opt_1/right_branch_opt_1.1.adm
@@ -0,0 +1,4 @@
+{ "t1nv": -1, "t2nv": -1, "t1x": { }, "t2x": { } }
+{ "t1nv": -2, "t2nv": -2, "t1x": { }, "t2x": { } }
+{ "t1nv": -3, "t1x": { } }
+{ "t1nv": -4, "t1x": { } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.10.adm
new file mode 100644
index 0000000..c5b431b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.10.adm
@@ -0,0 +1 @@
+50
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.11.adm
new file mode 100644
index 0000000..a040253
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan-select/push-limit-to-primary-scan-select.11.adm
@@ -0,0 +1,52 @@
+distribute result [$$128]
+-- DISTRIBUTE_RESULT |LOCAL|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |LOCAL|
+ aggregate [$$128] <- [agg-sql-sum($$129)]
+ -- AGGREGATE |LOCAL|
+ aggregate [$$129] <- [agg-sql-count(1)]
+ -- AGGREGATE |LOCAL|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ union
+ -- UNION_ALL |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 1000
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$126(ASC) ] |PARTITIONED|
+ limit 1000
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$126])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$126, $$onek1] <- test.onek1 condition (and(ge($$onek1.getField(2), 1), le($$onek1.getField(2), 10))) limit 1000
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 1000
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$127(ASC) ] |PARTITIONED|
+ limit 1000
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$127])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$127, $$onek1] <- test.onek1 condition (and(ge($$onek1.getField(2), -10), le($$onek1.getField(2), -1))) limit 1000
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.6.adm
new file mode 100644
index 0000000..97e3504
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.6.adm
@@ -0,0 +1 @@
+110
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.7.adm
new file mode 100644
index 0000000..f6de6e3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.7.adm
@@ -0,0 +1,52 @@
+distribute result [$$114]
+-- DISTRIBUTE_RESULT |LOCAL|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |LOCAL|
+ aggregate [$$114] <- [agg-sql-sum($$115)]
+ -- AGGREGATE |LOCAL|
+ aggregate [$$115] <- [agg-sql-count(1)]
+ -- AGGREGATE |LOCAL|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ union
+ -- UNION_ALL |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 100
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$112(ASC) ] |PARTITIONED|
+ limit 100
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$112])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$112, $$onek1] <- test.onek1 limit 100
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ limit 10
+ -- STREAM_LIMIT |UNPARTITIONED|
+ project ([])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- SORT_MERGE_EXCHANGE [$$113(ASC) ] |PARTITIONED|
+ limit 10
+ -- STREAM_LIMIT |PARTITIONED|
+ project ([$$113])
+ -- STREAM_PROJECT |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ data-scan []<-[$$113, $$onek1] <- test.onek1 limit 10
+ -- DATASOURCE_SCAN |PARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/list/get-item_03/get-item_03.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/get-item_03/get-item_03.3.adm
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/results/list/get-item_03/get-item_03.1.adm
rename to asterixdb/asterix-app/src/test/resources/runtimets/results/list/get-item_03/get-item_03.3.adm
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/list/get-item_03/get-item_03.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/get-item_03/get-item_03.4.adm
new file mode 100644
index 0000000..27ec71f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/get-item_03/get-item_03.4.adm
@@ -0,0 +1 @@
+{ "ta1": 3, "ta2": 2, "ta3": 1, "ta4": true, "tb1": 3, "tb2": 2, "tb3": 1, "tb4": true }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/list/var-in-list/var-in-list.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/var-in-list/var-in-list.03.adm
new file mode 100644
index 0000000..17ad9c9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/var-in-list/var-in-list.03.adm
@@ -0,0 +1,2 @@
+{ "list": [ { "id": 2, "f1": "bar" } ] }
+{ "list": [ { "id": 1, "f1": "foo" } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/list/var-in-list/var-in-list.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/var-in-list/var-in-list.04.adm
new file mode 100644
index 0000000..b875625
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/var-in-list/var-in-list.04.adm
@@ -0,0 +1,2 @@
+{ "list": { "id": 2, "f1": "bar" } }
+{ "list": { "id": 1, "f1": "foo" } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/list/var-in-list/var-in-list.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/var-in-list/var-in-list.05.adm
new file mode 100644
index 0000000..4721f1c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/var-in-list/var-in-list.05.adm
@@ -0,0 +1,2 @@
+{ "list": { "a": 5 } }
+{ "list": { "a": 7 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/list/var-in-list/var-in-list.06.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/var-in-list/var-in-list.06.adm
new file mode 100644
index 0000000..ff72349
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/list/var-in-list/var-in-list.06.adm
@@ -0,0 +1,2 @@
+{ "list": { "a": 5 } }
+{ "list": { "b": 7 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/load/csv_06/csv_06.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/load/csv_06/csv_06.1.adm
index e69de29..1cbcc6c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/load/csv_06/csv_06.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/load/csv_06/csv_06.1.adm
@@ -0,0 +1,3 @@
+{ "id": 2, "float": 0.2, "stringa": "test2a", "stringb": "test2b" }
+{ "id": 3, "float": 0.3, "stringa": "test,3a,3a,3a", "stringb": "\"\"test\"\"" }
+{ "id": 4, "float": 0.4, "stringa": "test\"4a\",4a", "stringb": " test with\nline break " }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/load/csv_07/csv_07.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/load/csv_07/csv_07.1.adm
index e69de29..1cbcc6c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/load/csv_07/csv_07.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/load/csv_07/csv_07.1.adm
@@ -0,0 +1,3 @@
+{ "id": 2, "float": 0.2, "stringa": "test2a", "stringb": "test2b" }
+{ "id": 3, "float": 0.3, "stringa": "test,3a,3a,3a", "stringb": "\"\"test\"\"" }
+{ "id": 4, "float": 0.4, "stringa": "test\"4a\",4a", "stringb": " test with\nline break " }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/metrics/external-dataset/external-dataset.2.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/metrics/external-dataset/external-dataset.2.regexadm
new file mode 100644
index 0000000..ae84a71
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/metrics/external-dataset/external-dataset.2.regexadm
@@ -0,0 +1 @@
+.*"processedObjects":15.*
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.2.adm
new file mode 100644
index 0000000..4b1e4bc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.2.adm
@@ -0,0 +1,4 @@
+true
+true
+true
+true
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.3.adm
new file mode 100644
index 0000000..5b24cb2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.3.adm
@@ -0,0 +1,18 @@
+distribute result [$$13]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ assign [$$13] <- [TRUE]
+ -- ASSIGN |UNPARTITIONED|
+ project ([])
+ -- STREAM_PROJECT |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ order (ASC, $$x)
+ -- STABLE_SORT [$$x(ASC)] |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ unnest $$x <- range(1, 4)
+ -- UNNEST |UNPARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.4.adm
new file mode 100644
index 0000000..5d73c0d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.4.adm
@@ -0,0 +1,4 @@
+true
+true
+false
+false
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.5.adm
new file mode 100644
index 0000000..d3f67c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.5.adm
@@ -0,0 +1,18 @@
+distribute result [$$13]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ project ([$$13])
+ -- STREAM_PROJECT |UNPARTITIONED|
+ assign [$$13] <- [le($$x, 2)]
+ -- ASSIGN |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ order (ASC, $$x)
+ -- STABLE_SORT [$$x(ASC)] |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ unnest $$x <- range(1, 4)
+ -- UNNEST |UNPARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.6.adm
new file mode 100644
index 0000000..77dc263
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.6.adm
@@ -0,0 +1,18 @@
+distribute result [$$13]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ project ([$$13])
+ -- STREAM_PROJECT |UNPARTITIONED|
+ assign [$$13] <- [or(null, le($$x, 2))]
+ -- ASSIGN |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ order (ASC, $$x)
+ -- STABLE_SORT [$$x(ASC)] |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ unnest $$x <- range(1, 4)
+ -- UNNEST |UNPARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.7.adm
new file mode 100644
index 0000000..0fe9557
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.7.adm
@@ -0,0 +1,18 @@
+distribute result [$$15]
+-- DISTRIBUTE_RESULT |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ project ([$$15])
+ -- STREAM_PROJECT |UNPARTITIONED|
+ assign [$$15] <- [or(TRUE, lt(get-year(current-date()), $$x))]
+ -- ASSIGN |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ order (ASC, $$x)
+ -- STABLE_SORT [$$x(ASC)] |UNPARTITIONED|
+ exchange
+ -- ONE_TO_ONE_EXCHANGE |UNPARTITIONED|
+ unnest $$x <- range(1, 4)
+ -- UNNEST |UNPARTITIONED|
+ empty-tuple-source
+ -- EMPTY_TUPLE_SOURCE |UNPARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-2886/query-ASTERIXDB-2886.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-2886/query-ASTERIXDB-2886.3.adm
new file mode 100644
index 0000000..7065d25
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-2886/query-ASTERIXDB-2886.3.adm
@@ -0,0 +1,4 @@
+{ "DataverseName": "test1", "DatasetName": "a", "IsAnonymous": true, "indexes": [ { "IndexName": "a" }, { "IndexName": "ia2" }, { "IndexName": "ia3" } ] }
+{ "DataverseName": "test1", "DatasetName": "b", "IsAnonymous": true, "indexes": [ { "IndexName": "b" }, { "IndexName": "ib2" }, { "IndexName": "ib3" } ] }
+{ "DataverseName": "test2", "DatasetName": "a", "IsAnonymous": true, "indexes": [ { "IndexName": "a" }, { "IndexName": "ia2" }, { "IndexName": "ia3" } ] }
+{ "DataverseName": "test2", "DatasetName": "b", "IsAnonymous": true, "indexes": [ { "IndexName": "b" }, { "IndexName": "ib2" }, { "IndexName": "ib3" } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/random/random.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/random/random.3.adm
new file mode 100644
index 0000000..5b9e73c4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/random/random.3.adm
@@ -0,0 +1,3 @@
+{ "i8": 0.26795443606823943, "i16": 0.26795443606823943, "i32": 0.26795443606823943, "i64": 0.26795443606823943, "float": 0.26795443606823943, "double": 0.26795443606823943 }
+{ "i8": 0.4533526797678967, "i16": 0.4533526797678967, "i32": 0.4533526797678967, "i64": 0.4533526797678967, "float": 0.4533526797678967, "double": 0.4533526797678967 }
+{ "i8": 0.38508513586474447, "i16": 0.38508513586474447, "i32": 0.38508513586474447, "i64": 0.38508513586474447, "float": 0.38508513586474447, "double": 0.38508513586474447 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/field-access/field-access.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/field-access/field-access.1.adm
index d7a20ef..f157d67 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/field-access/field-access.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/field-access/field-access.1.adm
@@ -1 +1,3 @@
-{ "null": null }
+{ "id": 0, "field_value": { "pid": 4 }, "is_missing": false }
+{ "id": 1, "is_missing": true }
+{ "id": 2, "field_value": null, "is_missing": false }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/field-access/field-access.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/field-access/field-access.2.adm
new file mode 100644
index 0000000..767c19b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/null-missing/field-access/field-access.2.adm
@@ -0,0 +1,3 @@
+{ "id": 0, "is_missing": true }
+{ "id": 1, "field_value": null, "is_missing": false }
+{ "id": 2, "field_value": 5, "is_missing": false }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/missing-fields/missing-fields.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/missing-fields/missing-fields.3.adm
new file mode 100644
index 0000000..ea6f909
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/missing-fields/missing-fields.3.adm
@@ -0,0 +1,3 @@
+[ { "field-name": "id", "field-type": "bigint", "is-open": false }, { "field-name": "f", "field-type": "object", "is-open": false } ]
+[ { "field-name": "id", "field-type": "bigint", "is-open": false }, { "field-name": "f", "field-type": "object", "is-open": false } ]
+[ { "field-name": "id", "field-type": "bigint", "is-open": false }, { "field-name": "f", "field-type": "object", "is-open": false, "nested": [ { "field-name": "inner_f", "field-type": "string", "is-open": false }, { "field-name": "inner_f2", "field-type": "object", "is-open": true, "nested": [ { "field-name": "f3", "field-type": "string", "is-open": true } ] } ] } ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/missing-fields/missing-fields.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/missing-fields/missing-fields.4.adm
new file mode 100644
index 0000000..0b364b1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/missing-fields/missing-fields.4.adm
@@ -0,0 +1,14 @@
+"CompactionPolicy"
+"CompactionPolicyProperties"
+"DatasetId"
+"DatasetName"
+"DatasetType"
+"DatatypeDataverseName"
+"DatatypeName"
+"DataverseName"
+"ExternalDetails"
+"GroupName"
+"Hints"
+"InternalDetails"
+"PendingOp"
+"Timestamp"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/tiny-social-example/tiny-social-example.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/tiny-social-example/tiny-social-example.4.adm
index 2938d8c..79f0217 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/tiny-social-example/tiny-social-example.4.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/tiny-social-example/tiny-social-example.4.adm
@@ -1 +1 @@
-[ { "field-name": "id", "field-type": "INT64", "is-open": false }, { "field-name": "alias", "field-type": "STRING", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": false }, { "field-name": "user-since", "field-type": "DATETIME", "is-open": false }, { "field-name": "friend-ids", "field-type": "UNORDEREDLIST", "is-open": false, "list": [ { "field-type": "INT64" } ] }, { "field-name": "employment", "field-type": "ORDEREDLIST", "is-open": false, "list": [ { "field-type": "RECORD", "nested": [ { "field-name": "organization-name", "field-type": "STRING", "is-open": false }, { "field-name": "start-date", "field-type": "DATE", "is-open": false }, { "field-name": "end-date", "field-type": "DATE", "is-open": false } ] } ] } ]
+[ { "field-name": "id", "field-type": "bigint", "is-open": false }, { "field-name": "alias", "field-type": "string", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": false }, { "field-name": "user-since", "field-type": "datetime", "is-open": false }, { "field-name": "friend-ids", "field-type": "multiset", "is-open": false, "list": [ { "field-type": "bigint" } ] }, { "field-name": "employment", "field-type": "array", "is-open": false, "list": [ { "field-type": "object", "nested": [ { "field-name": "organization-name", "field-type": "string", "is-open": false }, { "field-name": "start-date", "field-type": "date", "is-open": false }, { "field-name": "end-date", "field-type": "date", "is-open": false } ] } ] } ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/tiny-social-example/tiny-social-example.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/tiny-social-example/tiny-social-example.5.adm
index 74d2d3f..f8e1f55 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/tiny-social-example/tiny-social-example.5.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/tiny-social-example/tiny-social-example.5.adm
@@ -1,10 +1,10 @@
-[ { "field-name": "id", "field-type": "INT64", "is-open": false }, { "field-name": "alias", "field-type": "STRING", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": false }, { "field-name": "user-since", "field-type": "DATETIME", "is-open": false }, { "field-name": "friend-ids", "field-type": "UNORDEREDLIST", "is-open": false, "list": [ { "field-type": "INT64" }, { "field-type": "INT64" }, { "field-type": "INT64" }, { "field-type": "INT64" } ] }, { "field-name": "employment", "field-type": "ORDEREDLIST", "is-open": false, "list": [ { "field-type": "RECORD", "nested": [ { "field-name": "organization-name", "field-type": "STRING", "is-open": false }, { "field-name": "start-date", "field-type": "DATE", "is-open": false }, { "field-name": "end-date", "field-type": "DATE", "is-open": false } ] } ] } ]
-[ { "field-name": "id", "field-type": "INT64", "is-open": false }, { "field-name": "alias", "field-type": "STRING", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": false }, { "field-name": "user-since", "field-type": "DATETIME", "is-open": false }, { "field-name": "friend-ids", "field-type": "UNORDEREDLIST", "is-open": false, "list": [ { "field-type": "INT64" }, { "field-type": "INT64" } ] }, { "field-name": "employment", "field-type": "ORDEREDLIST", "is-open": false, "list": [ { "field-type": "RECORD", "nested": [ { "field-name": "organization-name", "field-type": "STRING", "is-open": false }, { "field-name": "start-date", "field-type": "DATE", "is-open": false }, { "field-name": "end-date", "field-type": "DATE", "is-open": false } ] } ] } ]
-[ { "field-name": "id", "field-type": "INT64", "is-open": false }, { "field-name": "alias", "field-type": "STRING", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": false }, { "field-name": "user-since", "field-type": "DATETIME", "is-open": false }, { "field-name": "friend-ids", "field-type": "UNORDEREDLIST", "is-open": false, "list": [ { "field-type": "INT64" }, { "field-type": "INT64" }, { "field-type": "INT64" }, { "field-type": "INT64" } ] }, { "field-name": "employment", "field-type": "ORDEREDLIST", "is-open": false, "list": [ { "field-type": "RECORD", "nested": [ { "field-name": "organization-name", "field-type": "STRING", "is-open": false }, { "field-name": "start-date", "field-type": "DATE", "is-open": false }, { "field-name": "end-date", "field-type": "DATE", "is-open": false } ] } ] } ]
-[ { "field-name": "id", "field-type": "INT64", "is-open": false }, { "field-name": "alias", "field-type": "STRING", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": false }, { "field-name": "user-since", "field-type": "DATETIME", "is-open": false }, { "field-name": "friend-ids", "field-type": "UNORDEREDLIST", "is-open": false, "list": [ { "field-type": "INT64" } ] }, { "field-name": "employment", "field-type": "ORDEREDLIST", "is-open": false, "list": [ { "field-type": "RECORD", "nested": [ { "field-name": "organization-name", "field-type": "STRING", "is-open": false }, { "field-name": "start-date", "field-type": "DATE", "is-open": false }, { "field-name": "end-date", "field-type": "DATE", "is-open": false } ] } ] } ]
-[ { "field-name": "id", "field-type": "INT64", "is-open": false }, { "field-name": "alias", "field-type": "STRING", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": false }, { "field-name": "user-since", "field-type": "DATETIME", "is-open": false }, { "field-name": "friend-ids", "field-type": "UNORDEREDLIST", "is-open": false, "list": [ { "field-type": "INT64" }, { "field-type": "INT64" }, { "field-type": "INT64" } ] }, { "field-name": "employment", "field-type": "ORDEREDLIST", "is-open": false, "list": [ { "field-type": "RECORD", "nested": [ { "field-name": "organization-name", "field-type": "STRING", "is-open": false }, { "field-name": "start-date", "field-type": "DATE", "is-open": false }, { "field-name": "end-date", "field-type": "DATE", "is-open": false } ] } ] } ]
-[ { "field-name": "id", "field-type": "INT64", "is-open": false }, { "field-name": "alias", "field-type": "STRING", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": false }, { "field-name": "user-since", "field-type": "DATETIME", "is-open": false }, { "field-name": "friend-ids", "field-type": "UNORDEREDLIST", "is-open": false, "list": [ { "field-type": "INT64" }, { "field-type": "INT64" }, { "field-type": "INT64" } ] }, { "field-name": "employment", "field-type": "ORDEREDLIST", "is-open": false, "list": [ { "field-type": "RECORD", "nested": [ { "field-name": "organization-name", "field-type": "STRING", "is-open": false }, { "field-name": "start-date", "field-type": "DATE", "is-open": false }, { "field-name": "end-date", "field-type": "DATE", "is-open": false } ] } ] } ]
-[ { "field-name": "id", "field-type": "INT64", "is-open": false }, { "field-name": "alias", "field-type": "STRING", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": false }, { "field-name": "user-since", "field-type": "DATETIME", "is-open": false }, { "field-name": "friend-ids", "field-type": "UNORDEREDLIST", "is-open": false, "list": [ { "field-type": "INT64" } ] }, { "field-name": "employment", "field-type": "ORDEREDLIST", "is-open": false, "list": [ { "field-type": "RECORD", "nested": [ { "field-name": "organization-name", "field-type": "STRING", "is-open": false }, { "field-name": "start-date", "field-type": "DATE", "is-open": false }, { "field-name": "end-date", "field-type": "DATE", "is-open": false } ] } ] } ]
-[ { "field-name": "id", "field-type": "INT64", "is-open": false }, { "field-name": "alias", "field-type": "STRING", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": false }, { "field-name": "user-since", "field-type": "DATETIME", "is-open": false }, { "field-name": "friend-ids", "field-type": "UNORDEREDLIST", "is-open": false, "list": [ { "field-type": "INT64" } ] }, { "field-name": "employment", "field-type": "ORDEREDLIST", "is-open": false, "list": [ { "field-type": "RECORD", "nested": [ { "field-name": "organization-name", "field-type": "STRING", "is-open": false }, { "field-name": "start-date", "field-type": "DATE", "is-open": false }, { "field-name": "end-date", "field-type": "DATE", "is-open": false } ] } ] } ]
-[ { "field-name": "id", "field-type": "INT64", "is-open": false }, { "field-name": "alias", "field-type": "STRING", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": false }, { "field-name": "user-since", "field-type": "DATETIME", "is-open": false }, { "field-name": "friend-ids", "field-type": "UNORDEREDLIST", "is-open": false, "list": [ { "field-type": "INT64" }, { "field-type": "INT64" } ] }, { "field-name": "employment", "field-type": "ORDEREDLIST", "is-open": false, "list": [ { "field-type": "RECORD", "nested": [ { "field-name": "organization-name", "field-type": "STRING", "is-open": false }, { "field-name": "start-date", "field-type": "DATE", "is-open": false }, { "field-name": "end-date", "field-type": "DATE", "is-open": false } ] } ] } ]
-[ { "field-name": "id", "field-type": "INT64", "is-open": false }, { "field-name": "alias", "field-type": "STRING", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": false }, { "field-name": "user-since", "field-type": "DATETIME", "is-open": false }, { "field-name": "friend-ids", "field-type": "UNORDEREDLIST", "is-open": false, "list": [ { "field-type": "INT64" }, { "field-type": "INT64" }, { "field-type": "INT64" } ] }, { "field-name": "employment", "field-type": "ORDEREDLIST", "is-open": false, "list": [ { "field-type": "RECORD", "nested": [ { "field-name": "organization-name", "field-type": "STRING", "is-open": false }, { "field-name": "start-date", "field-type": "DATE", "is-open": false }, { "field-name": "end-date", "field-type": "DATE", "is-open": false } ] } ] } ]
+[ { "field-name": "id", "field-type": "bigint", "is-open": false }, { "field-name": "alias", "field-type": "string", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": false }, { "field-name": "user-since", "field-type": "datetime", "is-open": false }, { "field-name": "friend-ids", "field-type": "multiset", "is-open": false, "list": [ { "field-type": "bigint" }, { "field-type": "bigint" }, { "field-type": "bigint" }, { "field-type": "bigint" } ] }, { "field-name": "employment", "field-type": "array", "is-open": false, "list": [ { "field-type": "object", "nested": [ { "field-name": "organization-name", "field-type": "string", "is-open": false }, { "field-name": "start-date", "field-type": "date", "is-open": false }, { "field-name": "end-date", "field-type": "date", "is-open": false } ] } ] } ]
+[ { "field-name": "id", "field-type": "bigint", "is-open": false }, { "field-name": "alias", "field-type": "string", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": false }, { "field-name": "user-since", "field-type": "datetime", "is-open": false }, { "field-name": "friend-ids", "field-type": "multiset", "is-open": false, "list": [ { "field-type": "bigint" }, { "field-type": "bigint" } ] }, { "field-name": "employment", "field-type": "array", "is-open": false, "list": [ { "field-type": "object", "nested": [ { "field-name": "organization-name", "field-type": "string", "is-open": false }, { "field-name": "start-date", "field-type": "date", "is-open": false }, { "field-name": "end-date", "field-type": "date", "is-open": false } ] } ] } ]
+[ { "field-name": "id", "field-type": "bigint", "is-open": false }, { "field-name": "alias", "field-type": "string", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": false }, { "field-name": "user-since", "field-type": "datetime", "is-open": false }, { "field-name": "friend-ids", "field-type": "multiset", "is-open": false, "list": [ { "field-type": "bigint" }, { "field-type": "bigint" }, { "field-type": "bigint" }, { "field-type": "bigint" } ] }, { "field-name": "employment", "field-type": "array", "is-open": false, "list": [ { "field-type": "object", "nested": [ { "field-name": "organization-name", "field-type": "string", "is-open": false }, { "field-name": "start-date", "field-type": "date", "is-open": false }, { "field-name": "end-date", "field-type": "date", "is-open": false } ] } ] } ]
+[ { "field-name": "id", "field-type": "bigint", "is-open": false }, { "field-name": "alias", "field-type": "string", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": false }, { "field-name": "user-since", "field-type": "datetime", "is-open": false }, { "field-name": "friend-ids", "field-type": "multiset", "is-open": false, "list": [ { "field-type": "bigint" } ] }, { "field-name": "employment", "field-type": "array", "is-open": false, "list": [ { "field-type": "object", "nested": [ { "field-name": "organization-name", "field-type": "string", "is-open": false }, { "field-name": "start-date", "field-type": "date", "is-open": false }, { "field-name": "end-date", "field-type": "date", "is-open": false } ] } ] } ]
+[ { "field-name": "id", "field-type": "bigint", "is-open": false }, { "field-name": "alias", "field-type": "string", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": false }, { "field-name": "user-since", "field-type": "datetime", "is-open": false }, { "field-name": "friend-ids", "field-type": "multiset", "is-open": false, "list": [ { "field-type": "bigint" }, { "field-type": "bigint" }, { "field-type": "bigint" } ] }, { "field-name": "employment", "field-type": "array", "is-open": false, "list": [ { "field-type": "object", "nested": [ { "field-name": "organization-name", "field-type": "string", "is-open": false }, { "field-name": "start-date", "field-type": "date", "is-open": false }, { "field-name": "end-date", "field-type": "date", "is-open": false } ] } ] } ]
+[ { "field-name": "id", "field-type": "bigint", "is-open": false }, { "field-name": "alias", "field-type": "string", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": false }, { "field-name": "user-since", "field-type": "datetime", "is-open": false }, { "field-name": "friend-ids", "field-type": "multiset", "is-open": false, "list": [ { "field-type": "bigint" }, { "field-type": "bigint" }, { "field-type": "bigint" } ] }, { "field-name": "employment", "field-type": "array", "is-open": false, "list": [ { "field-type": "object", "nested": [ { "field-name": "organization-name", "field-type": "string", "is-open": false }, { "field-name": "start-date", "field-type": "date", "is-open": false }, { "field-name": "end-date", "field-type": "date", "is-open": false } ] } ] } ]
+[ { "field-name": "id", "field-type": "bigint", "is-open": false }, { "field-name": "alias", "field-type": "string", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": false }, { "field-name": "user-since", "field-type": "datetime", "is-open": false }, { "field-name": "friend-ids", "field-type": "multiset", "is-open": false, "list": [ { "field-type": "bigint" } ] }, { "field-name": "employment", "field-type": "array", "is-open": false, "list": [ { "field-type": "object", "nested": [ { "field-name": "organization-name", "field-type": "string", "is-open": false }, { "field-name": "start-date", "field-type": "date", "is-open": false }, { "field-name": "end-date", "field-type": "date", "is-open": false } ] } ] } ]
+[ { "field-name": "id", "field-type": "bigint", "is-open": false }, { "field-name": "alias", "field-type": "string", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": false }, { "field-name": "user-since", "field-type": "datetime", "is-open": false }, { "field-name": "friend-ids", "field-type": "multiset", "is-open": false, "list": [ { "field-type": "bigint" } ] }, { "field-name": "employment", "field-type": "array", "is-open": false, "list": [ { "field-type": "object", "nested": [ { "field-name": "organization-name", "field-type": "string", "is-open": false }, { "field-name": "start-date", "field-type": "date", "is-open": false }, { "field-name": "end-date", "field-type": "date", "is-open": false } ] } ] } ]
+[ { "field-name": "id", "field-type": "bigint", "is-open": false }, { "field-name": "alias", "field-type": "string", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": false }, { "field-name": "user-since", "field-type": "datetime", "is-open": false }, { "field-name": "friend-ids", "field-type": "multiset", "is-open": false, "list": [ { "field-type": "bigint" }, { "field-type": "bigint" } ] }, { "field-name": "employment", "field-type": "array", "is-open": false, "list": [ { "field-type": "object", "nested": [ { "field-name": "organization-name", "field-type": "string", "is-open": false }, { "field-name": "start-date", "field-type": "date", "is-open": false }, { "field-name": "end-date", "field-type": "date", "is-open": false } ] } ] } ]
+[ { "field-name": "id", "field-type": "bigint", "is-open": false }, { "field-name": "alias", "field-type": "string", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": false }, { "field-name": "user-since", "field-type": "datetime", "is-open": false }, { "field-name": "friend-ids", "field-type": "multiset", "is-open": false, "list": [ { "field-type": "bigint" }, { "field-type": "bigint" }, { "field-type": "bigint" } ] }, { "field-name": "employment", "field-type": "array", "is-open": false, "list": [ { "field-type": "object", "nested": [ { "field-name": "organization-name", "field-type": "string", "is-open": false }, { "field-name": "start-date", "field-type": "date", "is-open": false }, { "field-name": "end-date", "field-type": "date", "is-open": false } ] } ] } ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/tiny-social-example/tiny-social-example.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/tiny-social-example/tiny-social-example.6.adm
index 73bfe63..9a60421 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/tiny-social-example/tiny-social-example.6.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/tiny-social-example/tiny-social-example.6.adm
@@ -1,15 +1,15 @@
-[ { "field-name": "message-id", "field-type": "INT64", "is-open": false }, { "field-name": "author-id", "field-type": "INT64", "is-open": false }, { "field-name": "in-response-to", "field-type": "INT64", "is-open": false }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "message", "field-type": "STRING", "is-open": false } ]
-[ { "field-name": "message-id", "field-type": "INT64", "is-open": false }, { "field-name": "author-id", "field-type": "INT64", "is-open": false }, { "field-name": "in-response-to", "field-type": "INT64", "is-open": false }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "message", "field-type": "STRING", "is-open": false } ]
-[ { "field-name": "message-id", "field-type": "INT64", "is-open": false }, { "field-name": "author-id", "field-type": "INT64", "is-open": false }, { "field-name": "in-response-to", "field-type": "INT64", "is-open": false }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "message", "field-type": "STRING", "is-open": false } ]
-[ { "field-name": "message-id", "field-type": "INT64", "is-open": false }, { "field-name": "author-id", "field-type": "INT64", "is-open": false }, { "field-name": "in-response-to", "field-type": "INT64", "is-open": false }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "message", "field-type": "STRING", "is-open": false } ]
-[ { "field-name": "message-id", "field-type": "INT64", "is-open": false }, { "field-name": "author-id", "field-type": "INT64", "is-open": false }, { "field-name": "in-response-to", "field-type": "INT64", "is-open": false }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "message", "field-type": "STRING", "is-open": false } ]
-[ { "field-name": "message-id", "field-type": "INT64", "is-open": false }, { "field-name": "author-id", "field-type": "INT64", "is-open": false }, { "field-name": "in-response-to", "field-type": "INT64", "is-open": false }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "message", "field-type": "STRING", "is-open": false } ]
-[ { "field-name": "message-id", "field-type": "INT64", "is-open": false }, { "field-name": "author-id", "field-type": "INT64", "is-open": false }, { "field-name": "in-response-to", "field-type": "INT64", "is-open": false }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "message", "field-type": "STRING", "is-open": false } ]
-[ { "field-name": "message-id", "field-type": "INT64", "is-open": false }, { "field-name": "author-id", "field-type": "INT64", "is-open": false }, { "field-name": "in-response-to", "field-type": "INT64", "is-open": false }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "message", "field-type": "STRING", "is-open": false } ]
-[ { "field-name": "message-id", "field-type": "INT64", "is-open": false }, { "field-name": "author-id", "field-type": "INT64", "is-open": false }, { "field-name": "in-response-to", "field-type": "INT64", "is-open": false }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "message", "field-type": "STRING", "is-open": false } ]
-[ { "field-name": "message-id", "field-type": "INT64", "is-open": false }, { "field-name": "author-id", "field-type": "INT64", "is-open": false }, { "field-name": "in-response-to", "field-type": "INT64", "is-open": false }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "message", "field-type": "STRING", "is-open": false } ]
-[ { "field-name": "message-id", "field-type": "INT64", "is-open": false }, { "field-name": "author-id", "field-type": "INT64", "is-open": false }, { "field-name": "in-response-to", "field-type": "INT64", "is-open": false }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "message", "field-type": "STRING", "is-open": false } ]
-[ { "field-name": "message-id", "field-type": "INT64", "is-open": false }, { "field-name": "author-id", "field-type": "INT64", "is-open": false }, { "field-name": "in-response-to", "field-type": "INT64", "is-open": false }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "message", "field-type": "STRING", "is-open": false } ]
-[ { "field-name": "message-id", "field-type": "INT64", "is-open": false }, { "field-name": "author-id", "field-type": "INT64", "is-open": false }, { "field-name": "in-response-to", "field-type": "INT64", "is-open": false }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "message", "field-type": "STRING", "is-open": false } ]
-[ { "field-name": "message-id", "field-type": "INT64", "is-open": false }, { "field-name": "author-id", "field-type": "INT64", "is-open": false }, { "field-name": "in-response-to", "field-type": "INT64", "is-open": false }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "message", "field-type": "STRING", "is-open": false } ]
-[ { "field-name": "message-id", "field-type": "INT64", "is-open": false }, { "field-name": "author-id", "field-type": "INT64", "is-open": false }, { "field-name": "in-response-to", "field-type": "INT64", "is-open": false }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "message", "field-type": "STRING", "is-open": false } ]
+[ { "field-name": "message-id", "field-type": "bigint", "is-open": false }, { "field-name": "author-id", "field-type": "bigint", "is-open": false }, { "field-name": "in-response-to", "field-type": "bigint", "is-open": false }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "message", "field-type": "string", "is-open": false } ]
+[ { "field-name": "message-id", "field-type": "bigint", "is-open": false }, { "field-name": "author-id", "field-type": "bigint", "is-open": false }, { "field-name": "in-response-to", "field-type": "bigint", "is-open": false }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "message", "field-type": "string", "is-open": false } ]
+[ { "field-name": "message-id", "field-type": "bigint", "is-open": false }, { "field-name": "author-id", "field-type": "bigint", "is-open": false }, { "field-name": "in-response-to", "field-type": "bigint", "is-open": false }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "message", "field-type": "string", "is-open": false } ]
+[ { "field-name": "message-id", "field-type": "bigint", "is-open": false }, { "field-name": "author-id", "field-type": "bigint", "is-open": false }, { "field-name": "in-response-to", "field-type": "bigint", "is-open": false }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "message", "field-type": "string", "is-open": false } ]
+[ { "field-name": "message-id", "field-type": "bigint", "is-open": false }, { "field-name": "author-id", "field-type": "bigint", "is-open": false }, { "field-name": "in-response-to", "field-type": "bigint", "is-open": false }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "message", "field-type": "string", "is-open": false } ]
+[ { "field-name": "message-id", "field-type": "bigint", "is-open": false }, { "field-name": "author-id", "field-type": "bigint", "is-open": false }, { "field-name": "in-response-to", "field-type": "bigint", "is-open": false }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "message", "field-type": "string", "is-open": false } ]
+[ { "field-name": "message-id", "field-type": "bigint", "is-open": false }, { "field-name": "author-id", "field-type": "bigint", "is-open": false }, { "field-name": "in-response-to", "field-type": "bigint", "is-open": false }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "message", "field-type": "string", "is-open": false } ]
+[ { "field-name": "message-id", "field-type": "bigint", "is-open": false }, { "field-name": "author-id", "field-type": "bigint", "is-open": false }, { "field-name": "in-response-to", "field-type": "bigint", "is-open": false }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "message", "field-type": "string", "is-open": false } ]
+[ { "field-name": "message-id", "field-type": "bigint", "is-open": false }, { "field-name": "author-id", "field-type": "bigint", "is-open": false }, { "field-name": "in-response-to", "field-type": "bigint", "is-open": false }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "message", "field-type": "string", "is-open": false } ]
+[ { "field-name": "message-id", "field-type": "bigint", "is-open": false }, { "field-name": "author-id", "field-type": "bigint", "is-open": false }, { "field-name": "in-response-to", "field-type": "bigint", "is-open": false }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "message", "field-type": "string", "is-open": false } ]
+[ { "field-name": "message-id", "field-type": "bigint", "is-open": false }, { "field-name": "author-id", "field-type": "bigint", "is-open": false }, { "field-name": "in-response-to", "field-type": "bigint", "is-open": false }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "message", "field-type": "string", "is-open": false } ]
+[ { "field-name": "message-id", "field-type": "bigint", "is-open": false }, { "field-name": "author-id", "field-type": "bigint", "is-open": false }, { "field-name": "in-response-to", "field-type": "bigint", "is-open": false }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "message", "field-type": "string", "is-open": false } ]
+[ { "field-name": "message-id", "field-type": "bigint", "is-open": false }, { "field-name": "author-id", "field-type": "bigint", "is-open": false }, { "field-name": "in-response-to", "field-type": "bigint", "is-open": false }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "message", "field-type": "string", "is-open": false } ]
+[ { "field-name": "message-id", "field-type": "bigint", "is-open": false }, { "field-name": "author-id", "field-type": "bigint", "is-open": false }, { "field-name": "in-response-to", "field-type": "bigint", "is-open": false }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "message", "field-type": "string", "is-open": false } ]
+[ { "field-name": "message-id", "field-type": "bigint", "is-open": false }, { "field-name": "author-id", "field-type": "bigint", "is-open": false }, { "field-name": "in-response-to", "field-type": "bigint", "is-open": false }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "message", "field-type": "string", "is-open": false } ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/tiny-social-example/tiny-social-example.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/tiny-social-example/tiny-social-example.7.adm
index 429d522..d63f06f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/tiny-social-example/tiny-social-example.7.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/tiny-social-example/tiny-social-example.7.adm
@@ -1,4 +1,4 @@
-[ { "field-name": "screen-name", "field-type": "STRING", "is-open": false }, { "field-name": "lang", "field-type": "STRING", "is-open": false }, { "field-name": "friends_count", "field-type": "INT64", "is-open": false }, { "field-name": "statuses_count", "field-type": "INT64", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": true }, { "field-name": "followers_count", "field-type": "INT64", "is-open": true } ]
-[ { "field-name": "screen-name", "field-type": "STRING", "is-open": false }, { "field-name": "lang", "field-type": "STRING", "is-open": false }, { "field-name": "friends_count", "field-type": "INT64", "is-open": false }, { "field-name": "statuses_count", "field-type": "INT64", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": true }, { "field-name": "followers_count", "field-type": "INT64", "is-open": true } ]
-[ { "field-name": "screen-name", "field-type": "STRING", "is-open": false }, { "field-name": "lang", "field-type": "STRING", "is-open": false }, { "field-name": "friends_count", "field-type": "INT64", "is-open": false }, { "field-name": "statuses_count", "field-type": "INT64", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": true }, { "field-name": "followers_count", "field-type": "INT64", "is-open": true } ]
-[ { "field-name": "screen-name", "field-type": "STRING", "is-open": false }, { "field-name": "lang", "field-type": "STRING", "is-open": false }, { "field-name": "friends_count", "field-type": "INT64", "is-open": false }, { "field-name": "statuses_count", "field-type": "INT64", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": true }, { "field-name": "followers_count", "field-type": "INT64", "is-open": true } ]
+[ { "field-name": "screen-name", "field-type": "string", "is-open": false }, { "field-name": "lang", "field-type": "string", "is-open": false }, { "field-name": "friends_count", "field-type": "bigint", "is-open": false }, { "field-name": "statuses_count", "field-type": "bigint", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": true }, { "field-name": "followers_count", "field-type": "bigint", "is-open": true } ]
+[ { "field-name": "screen-name", "field-type": "string", "is-open": false }, { "field-name": "lang", "field-type": "string", "is-open": false }, { "field-name": "friends_count", "field-type": "bigint", "is-open": false }, { "field-name": "statuses_count", "field-type": "bigint", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": true }, { "field-name": "followers_count", "field-type": "bigint", "is-open": true } ]
+[ { "field-name": "screen-name", "field-type": "string", "is-open": false }, { "field-name": "lang", "field-type": "string", "is-open": false }, { "field-name": "friends_count", "field-type": "bigint", "is-open": false }, { "field-name": "statuses_count", "field-type": "bigint", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": true }, { "field-name": "followers_count", "field-type": "bigint", "is-open": true } ]
+[ { "field-name": "screen-name", "field-type": "string", "is-open": false }, { "field-name": "lang", "field-type": "string", "is-open": false }, { "field-name": "friends_count", "field-type": "bigint", "is-open": false }, { "field-name": "statuses_count", "field-type": "bigint", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": true }, { "field-name": "followers_count", "field-type": "bigint", "is-open": true } ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/tiny-social-example/tiny-social-example.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/tiny-social-example/tiny-social-example.8.adm
index c31a69b..e84c545 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/tiny-social-example/tiny-social-example.8.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/tiny-social-example/tiny-social-example.8.adm
@@ -1,12 +1,12 @@
-[ { "field-name": "tweetid", "field-type": "STRING", "is-open": false }, { "field-name": "user", "field-type": "RECORD", "is-open": false, "nested": [ { "field-name": "screen-name", "field-type": "STRING", "is-open": false }, { "field-name": "lang", "field-type": "STRING", "is-open": false }, { "field-name": "friends_count", "field-type": "INT64", "is-open": false }, { "field-name": "statuses_count", "field-type": "INT64", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": true }, { "field-name": "followers_count", "field-type": "INT64", "is-open": true } ] }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "send-time", "field-type": "DATETIME", "is-open": false }, { "field-name": "referred-topics", "field-type": "UNORDEREDLIST", "is-open": false, "list": [ { "field-type": "STRING" }, { "field-type": "STRING" } ] }, { "field-name": "message-text", "field-type": "STRING", "is-open": false } ]
-[ { "field-name": "tweetid", "field-type": "STRING", "is-open": false }, { "field-name": "user", "field-type": "RECORD", "is-open": false, "nested": [ { "field-name": "screen-name", "field-type": "STRING", "is-open": false }, { "field-name": "lang", "field-type": "STRING", "is-open": false }, { "field-name": "friends_count", "field-type": "INT64", "is-open": false }, { "field-name": "statuses_count", "field-type": "INT64", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": true }, { "field-name": "followers_count", "field-type": "INT64", "is-open": true } ] }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "send-time", "field-type": "DATETIME", "is-open": false }, { "field-name": "referred-topics", "field-type": "UNORDEREDLIST", "is-open": false, "list": [ { "field-type": "STRING" }, { "field-type": "STRING" } ] }, { "field-name": "message-text", "field-type": "STRING", "is-open": false } ]
-[ { "field-name": "tweetid", "field-type": "STRING", "is-open": false }, { "field-name": "user", "field-type": "RECORD", "is-open": false, "nested": [ { "field-name": "screen-name", "field-type": "STRING", "is-open": false }, { "field-name": "lang", "field-type": "STRING", "is-open": false }, { "field-name": "friends_count", "field-type": "INT64", "is-open": false }, { "field-name": "statuses_count", "field-type": "INT64", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": true }, { "field-name": "followers_count", "field-type": "INT64", "is-open": true } ] }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "send-time", "field-type": "DATETIME", "is-open": false }, { "field-name": "referred-topics", "field-type": "UNORDEREDLIST", "is-open": false, "list": [ { "field-type": "STRING" }, { "field-type": "STRING" } ] }, { "field-name": "message-text", "field-type": "STRING", "is-open": false } ]
-[ { "field-name": "tweetid", "field-type": "STRING", "is-open": false }, { "field-name": "user", "field-type": "RECORD", "is-open": false, "nested": [ { "field-name": "screen-name", "field-type": "STRING", "is-open": false }, { "field-name": "lang", "field-type": "STRING", "is-open": false }, { "field-name": "friends_count", "field-type": "INT64", "is-open": false }, { "field-name": "statuses_count", "field-type": "INT64", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": true }, { "field-name": "followers_count", "field-type": "INT64", "is-open": true } ] }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "send-time", "field-type": "DATETIME", "is-open": false }, { "field-name": "referred-topics", "field-type": "UNORDEREDLIST", "is-open": false, "list": [ { "field-type": "STRING" }, { "field-type": "STRING" } ] }, { "field-name": "message-text", "field-type": "STRING", "is-open": false } ]
-[ { "field-name": "tweetid", "field-type": "STRING", "is-open": false }, { "field-name": "user", "field-type": "RECORD", "is-open": false, "nested": [ { "field-name": "screen-name", "field-type": "STRING", "is-open": false }, { "field-name": "lang", "field-type": "STRING", "is-open": false }, { "field-name": "friends_count", "field-type": "INT64", "is-open": false }, { "field-name": "statuses_count", "field-type": "INT64", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": true }, { "field-name": "followers_count", "field-type": "INT64", "is-open": true } ] }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "send-time", "field-type": "DATETIME", "is-open": false }, { "field-name": "referred-topics", "field-type": "UNORDEREDLIST", "is-open": false, "list": [ { "field-type": "STRING" }, { "field-type": "STRING" } ] }, { "field-name": "message-text", "field-type": "STRING", "is-open": false } ]
-[ { "field-name": "tweetid", "field-type": "STRING", "is-open": false }, { "field-name": "user", "field-type": "RECORD", "is-open": false, "nested": [ { "field-name": "screen-name", "field-type": "STRING", "is-open": false }, { "field-name": "lang", "field-type": "STRING", "is-open": false }, { "field-name": "friends_count", "field-type": "INT64", "is-open": false }, { "field-name": "statuses_count", "field-type": "INT64", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": true }, { "field-name": "followers_count", "field-type": "INT64", "is-open": true } ] }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "send-time", "field-type": "DATETIME", "is-open": false }, { "field-name": "referred-topics", "field-type": "UNORDEREDLIST", "is-open": false, "list": [ { "field-type": "STRING" }, { "field-type": "STRING" } ] }, { "field-name": "message-text", "field-type": "STRING", "is-open": false } ]
-[ { "field-name": "tweetid", "field-type": "STRING", "is-open": false }, { "field-name": "user", "field-type": "RECORD", "is-open": false, "nested": [ { "field-name": "screen-name", "field-type": "STRING", "is-open": false }, { "field-name": "lang", "field-type": "STRING", "is-open": false }, { "field-name": "friends_count", "field-type": "INT64", "is-open": false }, { "field-name": "statuses_count", "field-type": "INT64", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": true }, { "field-name": "followers_count", "field-type": "INT64", "is-open": true } ] }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "send-time", "field-type": "DATETIME", "is-open": false }, { "field-name": "referred-topics", "field-type": "UNORDEREDLIST", "is-open": false, "list": [ { "field-type": "STRING" }, { "field-type": "STRING" } ] }, { "field-name": "message-text", "field-type": "STRING", "is-open": false } ]
-[ { "field-name": "tweetid", "field-type": "STRING", "is-open": false }, { "field-name": "user", "field-type": "RECORD", "is-open": false, "nested": [ { "field-name": "screen-name", "field-type": "STRING", "is-open": false }, { "field-name": "lang", "field-type": "STRING", "is-open": false }, { "field-name": "friends_count", "field-type": "INT64", "is-open": false }, { "field-name": "statuses_count", "field-type": "INT64", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": true }, { "field-name": "followers_count", "field-type": "INT64", "is-open": true } ] }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "send-time", "field-type": "DATETIME", "is-open": false }, { "field-name": "referred-topics", "field-type": "UNORDEREDLIST", "is-open": false, "list": [ { "field-type": "STRING" }, { "field-type": "STRING" } ] }, { "field-name": "message-text", "field-type": "STRING", "is-open": false } ]
-[ { "field-name": "tweetid", "field-type": "STRING", "is-open": false }, { "field-name": "user", "field-type": "RECORD", "is-open": false, "nested": [ { "field-name": "screen-name", "field-type": "STRING", "is-open": false }, { "field-name": "lang", "field-type": "STRING", "is-open": false }, { "field-name": "friends_count", "field-type": "INT64", "is-open": false }, { "field-name": "statuses_count", "field-type": "INT64", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": true }, { "field-name": "followers_count", "field-type": "INT64", "is-open": true } ] }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "send-time", "field-type": "DATETIME", "is-open": false }, { "field-name": "referred-topics", "field-type": "UNORDEREDLIST", "is-open": false, "list": [ { "field-type": "STRING" }, { "field-type": "STRING" } ] }, { "field-name": "message-text", "field-type": "STRING", "is-open": false } ]
-[ { "field-name": "tweetid", "field-type": "STRING", "is-open": false }, { "field-name": "user", "field-type": "RECORD", "is-open": false, "nested": [ { "field-name": "screen-name", "field-type": "STRING", "is-open": false }, { "field-name": "lang", "field-type": "STRING", "is-open": false }, { "field-name": "friends_count", "field-type": "INT64", "is-open": false }, { "field-name": "statuses_count", "field-type": "INT64", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": true }, { "field-name": "followers_count", "field-type": "INT64", "is-open": true } ] }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "send-time", "field-type": "DATETIME", "is-open": false }, { "field-name": "referred-topics", "field-type": "UNORDEREDLIST", "is-open": false, "list": [ { "field-type": "STRING" }, { "field-type": "STRING" } ] }, { "field-name": "message-text", "field-type": "STRING", "is-open": false } ]
-[ { "field-name": "tweetid", "field-type": "STRING", "is-open": false }, { "field-name": "user", "field-type": "RECORD", "is-open": false, "nested": [ { "field-name": "screen-name", "field-type": "STRING", "is-open": false }, { "field-name": "lang", "field-type": "STRING", "is-open": false }, { "field-name": "friends_count", "field-type": "INT64", "is-open": false }, { "field-name": "statuses_count", "field-type": "INT64", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": true }, { "field-name": "followers_count", "field-type": "INT64", "is-open": true } ] }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "send-time", "field-type": "DATETIME", "is-open": false }, { "field-name": "referred-topics", "field-type": "UNORDEREDLIST", "is-open": false, "list": [ { "field-type": "STRING" }, { "field-type": "STRING" } ] }, { "field-name": "message-text", "field-type": "STRING", "is-open": false } ]
-[ { "field-name": "tweetid", "field-type": "STRING", "is-open": false }, { "field-name": "user", "field-type": "RECORD", "is-open": false, "nested": [ { "field-name": "screen-name", "field-type": "STRING", "is-open": false }, { "field-name": "lang", "field-type": "STRING", "is-open": false }, { "field-name": "friends_count", "field-type": "INT64", "is-open": false }, { "field-name": "statuses_count", "field-type": "INT64", "is-open": false }, { "field-name": "name", "field-type": "STRING", "is-open": true }, { "field-name": "followers_count", "field-type": "INT64", "is-open": true } ] }, { "field-name": "sender-location", "field-type": "POINT", "is-open": false }, { "field-name": "send-time", "field-type": "DATETIME", "is-open": false }, { "field-name": "referred-topics", "field-type": "UNORDEREDLIST", "is-open": false, "list": [ { "field-type": "STRING" }, { "field-type": "STRING" } ] }, { "field-name": "message-text", "field-type": "STRING", "is-open": false } ]
+[ { "field-name": "tweetid", "field-type": "string", "is-open": false }, { "field-name": "user", "field-type": "object", "is-open": false, "nested": [ { "field-name": "screen-name", "field-type": "string", "is-open": false }, { "field-name": "lang", "field-type": "string", "is-open": false }, { "field-name": "friends_count", "field-type": "bigint", "is-open": false }, { "field-name": "statuses_count", "field-type": "bigint", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": true }, { "field-name": "followers_count", "field-type": "bigint", "is-open": true } ] }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "send-time", "field-type": "datetime", "is-open": false }, { "field-name": "referred-topics", "field-type": "multiset", "is-open": false, "list": [ { "field-type": "string" }, { "field-type": "string" } ] }, { "field-name": "message-text", "field-type": "string", "is-open": false } ]
+[ { "field-name": "tweetid", "field-type": "string", "is-open": false }, { "field-name": "user", "field-type": "object", "is-open": false, "nested": [ { "field-name": "screen-name", "field-type": "string", "is-open": false }, { "field-name": "lang", "field-type": "string", "is-open": false }, { "field-name": "friends_count", "field-type": "bigint", "is-open": false }, { "field-name": "statuses_count", "field-type": "bigint", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": true }, { "field-name": "followers_count", "field-type": "bigint", "is-open": true } ] }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "send-time", "field-type": "datetime", "is-open": false }, { "field-name": "referred-topics", "field-type": "multiset", "is-open": false, "list": [ { "field-type": "string" }, { "field-type": "string" } ] }, { "field-name": "message-text", "field-type": "string", "is-open": false } ]
+[ { "field-name": "tweetid", "field-type": "string", "is-open": false }, { "field-name": "user", "field-type": "object", "is-open": false, "nested": [ { "field-name": "screen-name", "field-type": "string", "is-open": false }, { "field-name": "lang", "field-type": "string", "is-open": false }, { "field-name": "friends_count", "field-type": "bigint", "is-open": false }, { "field-name": "statuses_count", "field-type": "bigint", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": true }, { "field-name": "followers_count", "field-type": "bigint", "is-open": true } ] }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "send-time", "field-type": "datetime", "is-open": false }, { "field-name": "referred-topics", "field-type": "multiset", "is-open": false, "list": [ { "field-type": "string" }, { "field-type": "string" } ] }, { "field-name": "message-text", "field-type": "string", "is-open": false } ]
+[ { "field-name": "tweetid", "field-type": "string", "is-open": false }, { "field-name": "user", "field-type": "object", "is-open": false, "nested": [ { "field-name": "screen-name", "field-type": "string", "is-open": false }, { "field-name": "lang", "field-type": "string", "is-open": false }, { "field-name": "friends_count", "field-type": "bigint", "is-open": false }, { "field-name": "statuses_count", "field-type": "bigint", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": true }, { "field-name": "followers_count", "field-type": "bigint", "is-open": true } ] }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "send-time", "field-type": "datetime", "is-open": false }, { "field-name": "referred-topics", "field-type": "multiset", "is-open": false, "list": [ { "field-type": "string" }, { "field-type": "string" } ] }, { "field-name": "message-text", "field-type": "string", "is-open": false } ]
+[ { "field-name": "tweetid", "field-type": "string", "is-open": false }, { "field-name": "user", "field-type": "object", "is-open": false, "nested": [ { "field-name": "screen-name", "field-type": "string", "is-open": false }, { "field-name": "lang", "field-type": "string", "is-open": false }, { "field-name": "friends_count", "field-type": "bigint", "is-open": false }, { "field-name": "statuses_count", "field-type": "bigint", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": true }, { "field-name": "followers_count", "field-type": "bigint", "is-open": true } ] }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "send-time", "field-type": "datetime", "is-open": false }, { "field-name": "referred-topics", "field-type": "multiset", "is-open": false, "list": [ { "field-type": "string" }, { "field-type": "string" } ] }, { "field-name": "message-text", "field-type": "string", "is-open": false } ]
+[ { "field-name": "tweetid", "field-type": "string", "is-open": false }, { "field-name": "user", "field-type": "object", "is-open": false, "nested": [ { "field-name": "screen-name", "field-type": "string", "is-open": false }, { "field-name": "lang", "field-type": "string", "is-open": false }, { "field-name": "friends_count", "field-type": "bigint", "is-open": false }, { "field-name": "statuses_count", "field-type": "bigint", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": true }, { "field-name": "followers_count", "field-type": "bigint", "is-open": true } ] }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "send-time", "field-type": "datetime", "is-open": false }, { "field-name": "referred-topics", "field-type": "multiset", "is-open": false, "list": [ { "field-type": "string" }, { "field-type": "string" } ] }, { "field-name": "message-text", "field-type": "string", "is-open": false } ]
+[ { "field-name": "tweetid", "field-type": "string", "is-open": false }, { "field-name": "user", "field-type": "object", "is-open": false, "nested": [ { "field-name": "screen-name", "field-type": "string", "is-open": false }, { "field-name": "lang", "field-type": "string", "is-open": false }, { "field-name": "friends_count", "field-type": "bigint", "is-open": false }, { "field-name": "statuses_count", "field-type": "bigint", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": true }, { "field-name": "followers_count", "field-type": "bigint", "is-open": true } ] }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "send-time", "field-type": "datetime", "is-open": false }, { "field-name": "referred-topics", "field-type": "multiset", "is-open": false, "list": [ { "field-type": "string" }, { "field-type": "string" } ] }, { "field-name": "message-text", "field-type": "string", "is-open": false } ]
+[ { "field-name": "tweetid", "field-type": "string", "is-open": false }, { "field-name": "user", "field-type": "object", "is-open": false, "nested": [ { "field-name": "screen-name", "field-type": "string", "is-open": false }, { "field-name": "lang", "field-type": "string", "is-open": false }, { "field-name": "friends_count", "field-type": "bigint", "is-open": false }, { "field-name": "statuses_count", "field-type": "bigint", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": true }, { "field-name": "followers_count", "field-type": "bigint", "is-open": true } ] }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "send-time", "field-type": "datetime", "is-open": false }, { "field-name": "referred-topics", "field-type": "multiset", "is-open": false, "list": [ { "field-type": "string" }, { "field-type": "string" } ] }, { "field-name": "message-text", "field-type": "string", "is-open": false } ]
+[ { "field-name": "tweetid", "field-type": "string", "is-open": false }, { "field-name": "user", "field-type": "object", "is-open": false, "nested": [ { "field-name": "screen-name", "field-type": "string", "is-open": false }, { "field-name": "lang", "field-type": "string", "is-open": false }, { "field-name": "friends_count", "field-type": "bigint", "is-open": false }, { "field-name": "statuses_count", "field-type": "bigint", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": true }, { "field-name": "followers_count", "field-type": "bigint", "is-open": true } ] }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "send-time", "field-type": "datetime", "is-open": false }, { "field-name": "referred-topics", "field-type": "multiset", "is-open": false, "list": [ { "field-type": "string" }, { "field-type": "string" } ] }, { "field-name": "message-text", "field-type": "string", "is-open": false } ]
+[ { "field-name": "tweetid", "field-type": "string", "is-open": false }, { "field-name": "user", "field-type": "object", "is-open": false, "nested": [ { "field-name": "screen-name", "field-type": "string", "is-open": false }, { "field-name": "lang", "field-type": "string", "is-open": false }, { "field-name": "friends_count", "field-type": "bigint", "is-open": false }, { "field-name": "statuses_count", "field-type": "bigint", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": true }, { "field-name": "followers_count", "field-type": "bigint", "is-open": true } ] }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "send-time", "field-type": "datetime", "is-open": false }, { "field-name": "referred-topics", "field-type": "multiset", "is-open": false, "list": [ { "field-type": "string" }, { "field-type": "string" } ] }, { "field-name": "message-text", "field-type": "string", "is-open": false } ]
+[ { "field-name": "tweetid", "field-type": "string", "is-open": false }, { "field-name": "user", "field-type": "object", "is-open": false, "nested": [ { "field-name": "screen-name", "field-type": "string", "is-open": false }, { "field-name": "lang", "field-type": "string", "is-open": false }, { "field-name": "friends_count", "field-type": "bigint", "is-open": false }, { "field-name": "statuses_count", "field-type": "bigint", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": true }, { "field-name": "followers_count", "field-type": "bigint", "is-open": true } ] }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "send-time", "field-type": "datetime", "is-open": false }, { "field-name": "referred-topics", "field-type": "multiset", "is-open": false, "list": [ { "field-type": "string" }, { "field-type": "string" } ] }, { "field-name": "message-text", "field-type": "string", "is-open": false } ]
+[ { "field-name": "tweetid", "field-type": "string", "is-open": false }, { "field-name": "user", "field-type": "object", "is-open": false, "nested": [ { "field-name": "screen-name", "field-type": "string", "is-open": false }, { "field-name": "lang", "field-type": "string", "is-open": false }, { "field-name": "friends_count", "field-type": "bigint", "is-open": false }, { "field-name": "statuses_count", "field-type": "bigint", "is-open": false }, { "field-name": "name", "field-type": "string", "is-open": true }, { "field-name": "followers_count", "field-type": "bigint", "is-open": true } ] }, { "field-name": "sender-location", "field-type": "point", "is-open": false }, { "field-name": "send-time", "field-type": "datetime", "is-open": false }, { "field-name": "referred-topics", "field-type": "multiset", "is-open": false, "list": [ { "field-type": "string" }, { "field-type": "string" } ] }, { "field-name": "message-text", "field-type": "string", "is-open": false } ]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/tiny-social-example/tiny-social-example.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/tiny-social-example/tiny-social-example.9.adm
index 8693920..967135f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/tiny-social-example/tiny-social-example.9.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/get-object-fields/tiny-social-example/tiny-social-example.9.adm
@@ -1,6 +1,6 @@
-{ "count": 12, "field-name": "message-text", "field-type": "STRING" }
-{ "count": 12, "field-name": "referred-topics", "field-type": "UNORDEREDLIST" }
-{ "count": 12, "field-name": "send-time", "field-type": "DATETIME" }
-{ "count": 12, "field-name": "sender-location", "field-type": "POINT" }
-{ "count": 12, "field-name": "tweetid", "field-type": "STRING" }
-{ "count": 12, "field-name": "user", "field-type": "RECORD" }
+{ "count": 12, "field-name": "message-text", "field-type": "string" }
+{ "count": 12, "field-name": "referred-topics", "field-type": "multiset" }
+{ "count": 12, "field-name": "send-time", "field-type": "datetime" }
+{ "count": 12, "field-name": "sender-location", "field-type": "point" }
+{ "count": 12, "field-name": "tweetid", "field-type": "string" }
+{ "count": 12, "field-name": "user", "field-type": "object" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-non-enforced/index-selection/btree-index-composite-key-04/btree-index-composite-key-04.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-non-enforced/index-selection/btree-index-composite-key-04/btree-index-composite-key-04.3.adm
new file mode 100644
index 0000000..d8263ee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-non-enforced/index-selection/btree-index-composite-key-04/btree-index-composite-key-04.3.adm
@@ -0,0 +1 @@
+2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/index_01/index_01.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/index_01/index_01.3.adm
new file mode 100644
index 0000000..0881670
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/index_01/index_01.3.adm
@@ -0,0 +1,8 @@
+{ "id": 299, "fname": "Julio", "lname": "Iorio", "age": 37 }
+{ "id": 514, "fname": "Julio", "lname": "Ruben", "age": 41 }
+{ "id": 671, "fname": "Julio", "lname": "Vanpatten", "age": 30 }
+{ "id": 741, "fname": "Julio", "lname": "Mattocks", "age": 38 }
+{ "id": 881, "fname": "Julio", "lname": "Isa", "age": 38 }
+{ "id": 1418, "fname": "Julio", "lname": "Damore", "age": 27 }
+{ "id": 1430, "fname": "Julio", "lname": "Barkett", "age": 39 }
+{ "id": 8338, "fname": "Julio", "lname": "Bosket", "age": 28 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/index_01/index_01.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/index_01/index_01.4.adm
new file mode 100644
index 0000000..87c7773
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/index_01/index_01.4.adm
@@ -0,0 +1 @@
+{ "id": 881, "fname": "Julio", "lname": "Isa", "age": 38 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr-ASTERIXDB-2949/substr-ASTERIXDB-2949.0.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr-ASTERIXDB-2949/substr-ASTERIXDB-2949.0.adm
new file mode 100644
index 0000000..a36b551
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/string/substr-ASTERIXDB-2949/substr-ASTERIXDB-2949.0.adm
@@ -0,0 +1 @@
+{ "s": "•\tABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789\n•\tabcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ\tABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-2815/query-ASTERIXDB-2815.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-2815/query-ASTERIXDB-2815.3.adm
new file mode 100644
index 0000000..448de2d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-2815/query-ASTERIXDB-2815.3.adm
@@ -0,0 +1,2 @@
+{ "urls": [ "http://example.org/2003", "http://example.org/2004", "http://example.org/2009", "http://example.org/2010" ], "id": 2 }
+{ "urls": [ "http://example.org/2004", "http://example.org/2005", "http://example.org/2006", "http://example.org/2007", "http://example.org/2008", "http://example.org/2009" ], "id": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-2815/query-ASTERIXDB-2815.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-2815/query-ASTERIXDB-2815.4.adm
new file mode 100644
index 0000000..c35bf4e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-2815/query-ASTERIXDB-2815.4.adm
@@ -0,0 +1,5 @@
+{ "ranks": [ 1, 2 ], "id": 1 }
+{ "ranks": [ 1, 2, 3, 4 ], "id": 2 }
+{ "ranks": [ ], "id": 3 }
+{ "ranks": [ 1, 2, 3, 4, 5, 6 ], "id": 4 }
+{ "ranks": [ 1 ], "id": 5 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-2815/query-ASTERIXDB-2815.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-2815/query-ASTERIXDB-2815.5.adm
new file mode 100644
index 0000000..9ddd8aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-2815/query-ASTERIXDB-2815.5.adm
@@ -0,0 +1,5 @@
+{ "ranks": [ 1, 1 ], "id": 1 }
+{ "ranks": [ 1, 1, 2, 2 ], "id": 2 }
+{ "ranks": [ ], "id": 3 }
+{ "ranks": [ 1, 1, 2, 2, 3, 3 ], "id": 4 }
+{ "ranks": [ 1 ], "id": 5 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/types/any-object/any-object.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/types/any-object/any-object.2.adm
index c8f7e91..a35256d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/types/any-object/any-object.2.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/types/any-object/any-object.2.adm
@@ -1,3 +1,3 @@
{ "DataverseName": "test", "DatatypeName": "AnyObject", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "Record": { "IsOpen": true, "Fields": [ ] } }, "Timestamp": "Fri Mar 10 17:36:46 PST 2017" }
-{ "DataverseName": "test", "DatatypeName": "kv1", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false }, { "FieldName": "val", "FieldType": "AnyObject", "IsNullable": false } ] } }, "Timestamp": "Fri Mar 10 17:36:46 PST 2017" }
-{ "DataverseName": "test", "DatatypeName": "kv2", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false }, { "FieldName": "val", "FieldType": "AnyObject", "IsNullable": false } ] } }, "Timestamp": "Fri Mar 10 17:36:46 PST 2017" }
+{ "DataverseName": "test", "DatatypeName": "kv1", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "val", "FieldType": "AnyObject", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri Mar 10 17:36:46 PST 2017" }
+{ "DataverseName": "test", "DatatypeName": "kv2", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "id", "FieldType": "int32", "IsNullable": false, "IsMissable": false }, { "FieldName": "val", "FieldType": "AnyObject", "IsNullable": false, "IsMissable": false } ] } }, "Timestamp": "Fri Mar 10 17:36:46 PST 2017" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_nested/union_nested.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_nested/union_nested.3.adm
new file mode 100644
index 0000000..4737f9e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/union/union_nested/union_nested.3.adm
@@ -0,0 +1,138 @@
+{ "agg_sum": 25000000, "two": -1, "four": null, "ten": null, "twenty": null }
+{ "agg_sum": 12502500, "two": -1, "four": -3, "ten": null, "twenty": null }
+{ "agg_sum": 2504500, "two": -1, "four": -3, "ten": -9, "twenty": null }
+{ "agg_sum": 2504500, "two": -1, "four": -3, "ten": -9, "twenty": -19 }
+{ "agg_sum": 2498500, "two": -1, "four": -3, "ten": -7, "twenty": null }
+{ "agg_sum": 2498500, "two": -1, "four": -3, "ten": -7, "twenty": -7 }
+{ "agg_sum": 2502500, "two": -1, "four": -3, "ten": -5, "twenty": null }
+{ "agg_sum": 2502500, "two": -1, "four": -3, "ten": -5, "twenty": -15 }
+{ "agg_sum": 2496500, "two": -1, "four": -3, "ten": -3, "twenty": null }
+{ "agg_sum": 2496500, "two": -1, "four": -3, "ten": -3, "twenty": -3 }
+{ "agg_sum": 2500500, "two": -1, "four": -3, "ten": -1, "twenty": null }
+{ "agg_sum": 2500500, "two": -1, "four": -3, "ten": -1, "twenty": -11 }
+{ "agg_sum": 12497500, "two": -1, "four": -1, "ten": null, "twenty": null }
+{ "agg_sum": 2499500, "two": -1, "four": -1, "ten": -9, "twenty": null }
+{ "agg_sum": 2499500, "two": -1, "four": -1, "ten": -9, "twenty": -9 }
+{ "agg_sum": 2503500, "two": -1, "four": -1, "ten": -7, "twenty": null }
+{ "agg_sum": 2503500, "two": -1, "four": -1, "ten": -7, "twenty": -17 }
+{ "agg_sum": 2497500, "two": -1, "four": -1, "ten": -5, "twenty": null }
+{ "agg_sum": 2497500, "two": -1, "four": -1, "ten": -5, "twenty": -5 }
+{ "agg_sum": 2501500, "two": -1, "four": -1, "ten": -3, "twenty": null }
+{ "agg_sum": 2501500, "two": -1, "four": -1, "ten": -3, "twenty": -13 }
+{ "agg_sum": 2495500, "two": -1, "four": -1, "ten": -1, "twenty": null }
+{ "agg_sum": 2495500, "two": -1, "four": -1, "ten": -1, "twenty": -1 }
+{ "agg_sum": 24995000, "two": 0, "four": null, "ten": null, "twenty": null }
+{ "agg_sum": 24995000, "two": 0, "four": null, "ten": null, "twenty": null }
+{ "agg_sum": 12500000, "two": 0, "four": -2, "ten": null, "twenty": null }
+{ "agg_sum": 2504000, "two": 0, "four": -2, "ten": -8, "twenty": null }
+{ "agg_sum": 2504000, "two": 0, "four": -2, "ten": -8, "twenty": -18 }
+{ "agg_sum": 2498000, "two": 0, "four": -2, "ten": -6, "twenty": null }
+{ "agg_sum": 2498000, "two": 0, "four": -2, "ten": -6, "twenty": -6 }
+{ "agg_sum": 2502000, "two": 0, "four": -2, "ten": -4, "twenty": null }
+{ "agg_sum": 2502000, "two": 0, "four": -2, "ten": -4, "twenty": -14 }
+{ "agg_sum": 2496000, "two": 0, "four": -2, "ten": -2, "twenty": null }
+{ "agg_sum": 2496000, "two": 0, "four": -2, "ten": -2, "twenty": -2 }
+{ "agg_sum": 2500000, "two": 0, "four": -2, "ten": 0, "twenty": null }
+{ "agg_sum": 2500000, "two": 0, "four": -2, "ten": 0, "twenty": -10 }
+{ "agg_sum": 12495000, "two": 0, "four": 0, "ten": null, "twenty": null }
+{ "agg_sum": 12495000, "two": 0, "four": 0, "ten": null, "twenty": null }
+{ "agg_sum": 2499000, "two": 0, "four": 0, "ten": -8, "twenty": null }
+{ "agg_sum": 2499000, "two": 0, "four": 0, "ten": -8, "twenty": -8 }
+{ "agg_sum": 2503000, "two": 0, "four": 0, "ten": -6, "twenty": null }
+{ "agg_sum": 2503000, "two": 0, "four": 0, "ten": -6, "twenty": -16 }
+{ "agg_sum": 2497000, "two": 0, "four": 0, "ten": -4, "twenty": null }
+{ "agg_sum": 2497000, "two": 0, "four": 0, "ten": -4, "twenty": -4 }
+{ "agg_sum": 2501000, "two": 0, "four": 0, "ten": -2, "twenty": null }
+{ "agg_sum": 2501000, "two": 0, "four": 0, "ten": -2, "twenty": -12 }
+{ "agg_sum": 2495000, "two": 0, "four": 0, "ten": 0, "twenty": null }
+{ "agg_sum": 2495000, "two": 0, "four": 0, "ten": 0, "twenty": null }
+{ "agg_sum": 2495000, "two": 0, "four": 0, "ten": 0, "twenty": 0 }
+{ "agg_sum": 2495000, "two": 0, "four": 0, "ten": 0, "twenty": 0 }
+{ "agg_sum": 2501000, "two": 0, "four": 0, "ten": 2, "twenty": null }
+{ "agg_sum": 2501000, "two": 0, "four": 0, "ten": 2, "twenty": 12 }
+{ "agg_sum": 2497000, "two": 0, "four": 0, "ten": 4, "twenty": null }
+{ "agg_sum": 2497000, "two": 0, "four": 0, "ten": 4, "twenty": 4 }
+{ "agg_sum": 2503000, "two": 0, "four": 0, "ten": 6, "twenty": null }
+{ "agg_sum": 2503000, "two": 0, "four": 0, "ten": 6, "twenty": 16 }
+{ "agg_sum": 2499000, "two": 0, "four": 0, "ten": 8, "twenty": null }
+{ "agg_sum": 2499000, "two": 0, "four": 0, "ten": 8, "twenty": 8 }
+{ "agg_sum": 12500000, "two": 0, "four": 2, "ten": null, "twenty": null }
+{ "agg_sum": 2500000, "two": 0, "four": 2, "ten": 0, "twenty": null }
+{ "agg_sum": 2500000, "two": 0, "four": 2, "ten": 0, "twenty": 10 }
+{ "agg_sum": 2496000, "two": 0, "four": 2, "ten": 2, "twenty": null }
+{ "agg_sum": 2496000, "two": 0, "four": 2, "ten": 2, "twenty": 2 }
+{ "agg_sum": 2502000, "two": 0, "four": 2, "ten": 4, "twenty": null }
+{ "agg_sum": 2502000, "two": 0, "four": 2, "ten": 4, "twenty": 14 }
+{ "agg_sum": 2498000, "two": 0, "four": 2, "ten": 6, "twenty": null }
+{ "agg_sum": 2498000, "two": 0, "four": 2, "ten": 6, "twenty": 6 }
+{ "agg_sum": 2504000, "two": 0, "four": 2, "ten": 8, "twenty": null }
+{ "agg_sum": 2504000, "two": 0, "four": 2, "ten": 8, "twenty": 18 }
+{ "agg_sum": 25000000, "two": 1, "four": null, "ten": null, "twenty": null }
+{ "agg_sum": 12497500, "two": 1, "four": 1, "ten": null, "twenty": null }
+{ "agg_sum": 2495500, "two": 1, "four": 1, "ten": 1, "twenty": null }
+{ "agg_sum": 2495500, "two": 1, "four": 1, "ten": 1, "twenty": 1 }
+{ "agg_sum": 2501500, "two": 1, "four": 1, "ten": 3, "twenty": null }
+{ "agg_sum": 2501500, "two": 1, "four": 1, "ten": 3, "twenty": 13 }
+{ "agg_sum": 2497500, "two": 1, "four": 1, "ten": 5, "twenty": null }
+{ "agg_sum": 2497500, "two": 1, "four": 1, "ten": 5, "twenty": 5 }
+{ "agg_sum": 2503500, "two": 1, "four": 1, "ten": 7, "twenty": null }
+{ "agg_sum": 2503500, "two": 1, "four": 1, "ten": 7, "twenty": 17 }
+{ "agg_sum": 2499500, "two": 1, "four": 1, "ten": 9, "twenty": null }
+{ "agg_sum": 2499500, "two": 1, "four": 1, "ten": 9, "twenty": 9 }
+{ "agg_sum": 12502500, "two": 1, "four": 3, "ten": null, "twenty": null }
+{ "agg_sum": 2500500, "two": 1, "four": 3, "ten": 1, "twenty": null }
+{ "agg_sum": 2500500, "two": 1, "four": 3, "ten": 1, "twenty": 11 }
+{ "agg_sum": 2496500, "two": 1, "four": 3, "ten": 3, "twenty": null }
+{ "agg_sum": 2496500, "two": 1, "four": 3, "ten": 3, "twenty": 3 }
+{ "agg_sum": 2502500, "two": 1, "four": 3, "ten": 5, "twenty": null }
+{ "agg_sum": 2502500, "two": 1, "four": 3, "ten": 5, "twenty": 15 }
+{ "agg_sum": 2498500, "two": 1, "four": 3, "ten": 7, "twenty": null }
+{ "agg_sum": 2498500, "two": 1, "four": 3, "ten": 7, "twenty": 7 }
+{ "agg_sum": 2504500, "two": 1, "four": 3, "ten": 9, "twenty": null }
+{ "agg_sum": 2504500, "two": 1, "four": 3, "ten": 9, "twenty": 19 }
+{ "agg_sum": 24995000, "two": "0", "four": null, "ten": null, "twenty": null }
+{ "agg_sum": 12495000, "two": "0", "four": "0", "ten": null, "twenty": null }
+{ "agg_sum": 2495000, "two": "0", "four": "0", "ten": "0", "twenty": null }
+{ "agg_sum": 2495000, "two": "0", "four": "0", "ten": "0", "twenty": "0" }
+{ "agg_sum": 2501000, "two": "0", "four": "0", "ten": "2", "twenty": null }
+{ "agg_sum": 2501000, "two": "0", "four": "0", "ten": "2", "twenty": "12" }
+{ "agg_sum": 2497000, "two": "0", "four": "0", "ten": "4", "twenty": null }
+{ "agg_sum": 2497000, "two": "0", "four": "0", "ten": "4", "twenty": "4" }
+{ "agg_sum": 2503000, "two": "0", "four": "0", "ten": "6", "twenty": null }
+{ "agg_sum": 2503000, "two": "0", "four": "0", "ten": "6", "twenty": "16" }
+{ "agg_sum": 2499000, "two": "0", "four": "0", "ten": "8", "twenty": null }
+{ "agg_sum": 2499000, "two": "0", "four": "0", "ten": "8", "twenty": "8" }
+{ "agg_sum": 12500000, "two": "0", "four": "2", "ten": null, "twenty": null }
+{ "agg_sum": 2500000, "two": "0", "four": "2", "ten": "0", "twenty": null }
+{ "agg_sum": 2500000, "two": "0", "four": "2", "ten": "0", "twenty": "10" }
+{ "agg_sum": 2496000, "two": "0", "four": "2", "ten": "2", "twenty": null }
+{ "agg_sum": 2496000, "two": "0", "four": "2", "ten": "2", "twenty": "2" }
+{ "agg_sum": 2502000, "two": "0", "four": "2", "ten": "4", "twenty": null }
+{ "agg_sum": 2502000, "two": "0", "four": "2", "ten": "4", "twenty": "14" }
+{ "agg_sum": 2498000, "two": "0", "four": "2", "ten": "6", "twenty": null }
+{ "agg_sum": 2498000, "two": "0", "four": "2", "ten": "6", "twenty": "6" }
+{ "agg_sum": 2504000, "two": "0", "four": "2", "ten": "8", "twenty": null }
+{ "agg_sum": 2504000, "two": "0", "four": "2", "ten": "8", "twenty": "18" }
+{ "agg_sum": 25000000, "two": "1", "four": null, "ten": null, "twenty": null }
+{ "agg_sum": 12497500, "two": "1", "four": "1", "ten": null, "twenty": null }
+{ "agg_sum": 2495500, "two": "1", "four": "1", "ten": "1", "twenty": null }
+{ "agg_sum": 2495500, "two": "1", "four": "1", "ten": "1", "twenty": "1" }
+{ "agg_sum": 2501500, "two": "1", "four": "1", "ten": "3", "twenty": null }
+{ "agg_sum": 2501500, "two": "1", "four": "1", "ten": "3", "twenty": "13" }
+{ "agg_sum": 2497500, "two": "1", "four": "1", "ten": "5", "twenty": null }
+{ "agg_sum": 2497500, "two": "1", "four": "1", "ten": "5", "twenty": "5" }
+{ "agg_sum": 2503500, "two": "1", "four": "1", "ten": "7", "twenty": null }
+{ "agg_sum": 2503500, "two": "1", "four": "1", "ten": "7", "twenty": "17" }
+{ "agg_sum": 2499500, "two": "1", "four": "1", "ten": "9", "twenty": null }
+{ "agg_sum": 2499500, "two": "1", "four": "1", "ten": "9", "twenty": "9" }
+{ "agg_sum": 12502500, "two": "1", "four": "3", "ten": null, "twenty": null }
+{ "agg_sum": 2500500, "two": "1", "four": "3", "ten": "1", "twenty": null }
+{ "agg_sum": 2500500, "two": "1", "four": "3", "ten": "1", "twenty": "11" }
+{ "agg_sum": 2496500, "two": "1", "four": "3", "ten": "3", "twenty": null }
+{ "agg_sum": 2496500, "two": "1", "four": "3", "ten": "3", "twenty": "3" }
+{ "agg_sum": 2502500, "two": "1", "four": "3", "ten": "5", "twenty": null }
+{ "agg_sum": 2502500, "two": "1", "four": "3", "ten": "5", "twenty": "15" }
+{ "agg_sum": 2498500, "two": "1", "four": "3", "ten": "7", "twenty": null }
+{ "agg_sum": 2498500, "two": "1", "four": "3", "ten": "7", "twenty": "7" }
+{ "agg_sum": 2504500, "two": "1", "four": "3", "ten": "9", "twenty": null }
+{ "agg_sum": 2504500, "two": "1", "four": "3", "ten": "9", "twenty": "19" }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/unnest/ASTERIXDB-2750_unnest_join/ASTERIXDB-2750_unnest_join.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/unnest/ASTERIXDB-2750_unnest_join/ASTERIXDB-2750_unnest_join.1.adm
new file mode 100644
index 0000000..cabf43b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/unnest/ASTERIXDB-2750_unnest_join/ASTERIXDB-2750_unnest_join.1.adm
@@ -0,0 +1 @@
+24
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/unnest/ASTERIXDB-2750_unnest_join/ASTERIXDB-2750_unnest_join.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/unnest/ASTERIXDB-2750_unnest_join/ASTERIXDB-2750_unnest_join.2.adm
new file mode 100644
index 0000000..8bc6583
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/unnest/ASTERIXDB-2750_unnest_join/ASTERIXDB-2750_unnest_join.2.adm
@@ -0,0 +1 @@
+120
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/unnest/ASTERIXDB-2844_unnest_syntax/ASTERIXDB-2844_unnest_syntax.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/unnest/ASTERIXDB-2844_unnest_syntax/ASTERIXDB-2844_unnest_syntax.1.adm
new file mode 100644
index 0000000..6b71814
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/unnest/ASTERIXDB-2844_unnest_syntax/ASTERIXDB-2844_unnest_syntax.1.adm
@@ -0,0 +1,4 @@
+{ "t": { "r": 1, "ra": [ 10, 11 ] }, "ra": 10 }
+{ "t": { "r": 1, "ra": [ 10, 11 ] }, "ra": 11 }
+{ "t": { "r": 2, "ra": [ 20, 21 ] }, "ra": 20 }
+{ "t": { "r": 2, "ra": [ 20, 21 ] }, "ra": 21 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/window/misc_01/misc_01.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/misc_01/misc_01.10.adm
new file mode 100644
index 0000000..8e98bdc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/window/misc_01/misc_01.10.adm
@@ -0,0 +1,4 @@
+{ "q1": [ { "nth": 3 }, { "nth": 3 }, { "nth": 3 }, { "nth": 3 } ], "c2": 1 }
+{ "q1": [ { "nth": 3 }, { "nth": 3 }, { "nth": 3 }, { "nth": 3 } ], "c2": 2 }
+{ "q1": [ { "nth": 3 }, { "nth": 3 }, { "nth": 3 }, { "nth": 3 } ], "c2": 3 }
+{ "q1": [ { "nth": 3 }, { "nth": 3 }, { "nth": 3 }, { "nth": 3 } ], "c2": 4 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/avg_mixed/avg_mixed.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/avg_mixed/avg_mixed.2.ast
deleted file mode 100644
index e69de29..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/avg_mixed/avg_mixed.2.ast
+++ /dev/null
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/avg_mixed/avg_mixed.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/avg_mixed/avg_mixed.3.ast
index 034df11..7fac994 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/avg_mixed/avg_mixed.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-sql/avg_mixed/avg_mixed.3.ast
@@ -10,7 +10,7 @@
LiteralExpr [STRING] [2.0]
]
LiteralExpr [STRING] [hello world]
- LiteralExpr [LONG] [93847382783847382]
+ LiteralExpr [LONG] [10]
FunctionCall null.date@1[
LiteralExpr [STRING] [2013-01-01]
]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate/avg_mixed/avg_mixed.2.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate/avg_mixed/avg_mixed.2.ast
deleted file mode 100644
index e69de29..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate/avg_mixed/avg_mixed.2.ast
+++ /dev/null
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/cross-dataverse/cross-dv01/cross-dv01.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/cross-dataverse/cross-dv01/cross-dv01.1.ast
index c3a4877..ffe5bc3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/cross-dataverse/cross-dv01/cross-dv01.1.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/cross-dataverse/cross-dv01/cross-dv01.1.ast
@@ -17,7 +17,7 @@
dept : string
}
]
-DatasetDecl ugdstd(stdType) partitioned by [[id]]
-DatasetDecl gdstd(stdType) partitioned by [[id]]
+DatasetDecl ugdstd(student.stdType) partitioned by [[id]]
+DatasetDecl gdstd(student.stdType) partitioned by [[id]]
DatasetDecl prof(tchrType) partitioned by [[id]]
-DatasetDecl pstdoc(tchrType) partitioned by [[id]]
+DatasetDecl pstdoc(tchrType) partitioned by [[id]]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/add_double/add_double.1.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/add_double/add_double.1.ast
index 4550cfe..7e1443c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/add_double/add_double.1.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/numeric/add_double/add_double.1.ast
@@ -97,12 +97,7 @@
LiteralExpr [STRING] [-6.5d]
]
+
- IndexAccessor [
- OrderedListConstructor [
- LiteralExpr [DOUBLE] [1.0]
- ]
- Index: - LiteralExpr [LONG] [1]
- ]
+ LiteralExpr [MISSING]
]
)
]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/window/misc_01/misc_01.10.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/window/misc_01/misc_01.10.ast
new file mode 100644
index 0000000..368f261
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/window/misc_01/misc_01.10.ast
@@ -0,0 +1,50 @@
+DataverseUse test
+Query:
+SELECT [
+FieldAccessor [
+ Variable [ Name=$y ]
+ Field=c2
+]
+c2
+(
+ SELECT [
+ WINDOW test.nth_value@2[
+ Variable [ Name=$c2 ]
+ LiteralExpr [LONG] [3]
+ ]
+ OVER (
+ PARTITION BY
+ FieldAccessor [
+ Variable [ Name=$x ]
+ Field=one
+ ]
+ ORDER BY
+ FieldAccessor [
+ Variable [ Name=$x ]
+ Field=c2
+ ]
+ ASC
+ range between unbounded preceding and unbounded following exclude no others
+ )
+ nth
+ ]
+ FROM [ FunctionCall asterix.dataset@1[
+ LiteralExpr [STRING] [test.t1]
+ ]
+ AS Variable [ Name=$x ]
+ ]
+)
+q1
+]
+FROM [ FunctionCall asterix.dataset@1[
+ LiteralExpr [STRING] [test.t1]
+ ]
+ AS Variable [ Name=$y ]
+]
+Orderby
+ FieldAccessor [
+ Variable [ Name=$y ]
+ Field=c2
+ ]
+ ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
index f16b187..cfb2e5d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -1679,19 +1679,16 @@
<test-case FilePath="load">
<compilation-unit name="csv_05"><!-- Someone should check and verify -->
<output-dir compare="Text">csv_05</output-dir>
- <expected-error>At record: 1</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="load">
<compilation-unit name="csv_06"><!-- Someone should check and verify -->
<output-dir compare="Text">csv_06</output-dir>
- <expected-error>At record: 1</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="load">
<compilation-unit name="csv_07"><!-- Someone should check and verify -->
<output-dir compare="Text">csv_07</output-dir>
- <expected-error>At record: 1</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="load">
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset.xml
new file mode 100644
index 0000000..5df2549
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset.xml
@@ -0,0 +1,267 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements. See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership. The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License. You may obtain a copy of the License at
+ !
+ ! http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied. See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp">
+ <test-group name="external-dataset">
+ <test-case FilePath="external-dataset/aws/s3/">
+ <compilation-unit name="custom-buffer-size">
+ <output-dir compare="Text">custom-buffer-size</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/json/json">
+ <output-dir compare="Text">aws/s3/json/json</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/json/gz">
+ <output-dir compare="Text">aws/s3/json/gz</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/json/mixed">
+ <output-dir compare="Text">aws/s3/json/mixed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/csv/csv">
+ <output-dir compare="Text">aws/s3/csv/csv</output-dir>
+ </compilation-unit>
+ </test-case><test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/csv/gz">
+ <output-dir compare="Text">aws/s3/csv/gz</output-dir>
+ </compilation-unit>
+ </test-case><test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/csv/mixed">
+ <output-dir compare="Text">aws/s3/csv/mixed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/tsv/tsv">
+ <output-dir compare="Text">aws/s3/tsv/tsv</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/tsv/gz">
+ <output-dir compare="Text">aws/s3/tsv/gz</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/tsv/mixed">
+ <output-dir compare="Text">aws/s3/tsv/mixed</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/empty-string-definition">
+ <output-dir compare="Text">aws/s3/empty-string-definition</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/over-1000-objects">
+ <output-dir compare="Text">aws/s3/over-1000-objects</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/malformed-json">
+ <output-dir compare="Text">aws/s3/malformed-json</output-dir>
+ <expected-error>Parsing error at malformed-data/duplicate-fields.json line 1 field field: Duplicate field 'field'</expected-error>
+ <expected-error>Parsing error at malformed-data/malformed-json.json line 1 field field: Unexpected character ('}' (code 125)): was expecting double-quote to start field name</expected-error>
+ <expected-error>Parsing error at malformed-data/malformed-json-2.json line 4 field array_f: Unexpected character (']' (code 93)): expected a valid value (JSON String, Number, Array, Object or token 'null', 'true' or 'false')</expected-error>
+ <expected-error>Parsing error at malformed-data/malformed-jsonl-1.json line 3 field field2: Unrecognized token 'truee': was expecting (JSON String, Number, Array, Object or token 'null', 'true' or 'false')</expected-error>
+ <expected-error>Parsing error at malformed-data/malformed-jsonl-2.json line 11 field array_f: Unexpected character (']' (code 93)): expected a valid value (JSON String, Number, Array, Object or token 'null', 'true' or 'false')</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/definition-does-not-exist">
+ <output-dir compare="Text">aws/s3/definition-does-not-exist</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/invalid-endpoint">
+ <output-dir compare="Text">aws/s3/invalid-endpoint</output-dir>
+ <expected-error>External source error. Invalid service endpoint ^invalid-endpoint^</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/bucket-does-not-exist">
+ <output-dir compare="Text">aws/s3/bucket-does-not-exist</output-dir>
+ <expected-error>External source error. The specified bucket does not exist (Service: S3, Status Code: 404, Request ID: null, Extended Request ID: null)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset" check-warnings="true">
+ <compilation-unit name="aws/s3/no-files-returned/definition-points-to-nothing">
+ <output-dir compare="Text">aws/s3/no-files-returned/definition-points-to-nothing</output-dir>
+ <source-location>false</source-location>
+ <expected-warn>The provided external dataset configuration returned no files from the external source</expected-warn>
+ <expected-warn>The provided external dataset configuration returned no files from the external source</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset" check-warnings="true">
+ <compilation-unit name="aws/s3/no-files-returned/exclude-all-files">
+ <output-dir compare="Text">aws/s3/no-files-returned/exclude-all-files</output-dir>
+ <expected-warn>The provided external dataset configuration returned no files from the external source</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset" check-warnings="true">
+ <compilation-unit name="aws/s3/no-files-returned/include-no-files">
+ <output-dir compare="Text">aws/s3/no-files-returned/include-no-files</output-dir>
+ <expected-warn>The provided external dataset configuration returned no files from the external source</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/non-s3-region">
+ <output-dir compare="Text">aws/s3/non-s3-region</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+ <test-group name="include-exclude">
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/include-exclude/bad-name-1">
+ <output-dir compare="Text">aws/s3/include-exclude/bad-name-1</output-dir>
+ <expected-error>Invalid format for property "exclude1"</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/include-exclude/bad-name-2">
+ <output-dir compare="Text">aws/s3/include-exclude/bad-name-2</output-dir>
+ <expected-error>Invalid format for property "exclude#"</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/include-exclude/bad-name-3">
+ <output-dir compare="Text">aws/s3/include-exclude/bad-name-3</output-dir>
+ <expected-error>Invalid format for property "exclude#hello"</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/include-exclude/both">
+ <output-dir compare="Text">aws/s3/include-exclude/both</output-dir>
+ <expected-error>The parameters "include" and "exclude" cannot be provided at the same time</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/include-exclude/exclude-all">
+ <output-dir compare="Text">aws/s3/include-exclude/exclude-all</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/include-exclude/exclude-1">
+ <output-dir compare="Text">aws/s3/include-exclude/exclude-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/include-exclude/exclude-2">
+ <output-dir compare="Text">aws/s3/include-exclude/exclude-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/include-exclude/exclude-3">
+ <output-dir compare="Text">aws/s3/include-exclude/exclude-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/include-exclude/exclude-4">
+ <output-dir compare="Text">aws/s3/include-exclude/exclude-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/include-exclude/exclude-5">
+ <output-dir compare="Text">aws/s3/include-exclude/exclude-5</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/include-exclude/exclude-6">
+ <output-dir compare="Text">aws/s3/include-exclude/exclude-6</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/include-exclude/include-all">
+ <output-dir compare="Text">aws/s3/include-exclude/include-all</output-dir>
+ <expected-error>Malformed input stream</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/include-exclude/include-1">
+ <output-dir compare="Text">aws/s3/include-exclude/include-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/include-exclude/include-2">
+ <output-dir compare="Text">aws/s3/include-exclude/include-2</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/include-exclude/include-3">
+ <output-dir compare="Text">aws/s3/include-exclude/include-3</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/include-exclude/include-4">
+ <output-dir compare="Text">aws/s3/include-exclude/include-4</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/include-exclude/include-5">
+ <output-dir compare="Text">aws/s3/include-exclude/include-5</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/include-exclude/include-6">
+ <output-dir compare="Text">aws/s3/include-exclude/include-6</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/include-exclude/include-7">
+ <output-dir compare="Text">aws/s3/include-exclude/include-7</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/include-exclude/include-8">
+ <output-dir compare="Text">aws/s3/include-exclude/include-8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/include-exclude/include-9">
+ <output-dir compare="Text">aws/s3/include-exclude/include-9</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/include-exclude/include-10">
+ <output-dir compare="Text">aws/s3/include-exclude/include-10</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/include-exclude/include-11">
+ <output-dir compare="Text">aws/s3/include-exclude/include-11</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/include-exclude/include-12">
+ <output-dir compare="Text">aws/s3/include-exclude/include-12</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset">
+ <compilation-unit name="aws/s3/anonymous_no_auth">
+ <output-dir compare="Text">anonymous_no_auth</output-dir>
+ <expected-error>ASX3119: Parameter 'secretAccessKey' is required if 'accessKeyId' is provided</expected-error>
+ <expected-error>ASX3119: Parameter 'accessKeyId' is required if 'secretAccessKey' is provided</expected-error>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+</test-suite>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_one_partition.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_one_partition.xml
new file mode 100644
index 0000000..6704d78
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_external_dataset_one_partition.xml
@@ -0,0 +1,91 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements. See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership. The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License. You may obtain a copy of the License at
+ !
+ ! http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied. See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp">
+ <test-group name="external-dataset">
+ <test-case FilePath="external-dataset" check-warnings="true">
+ <compilation-unit name="aws/s3/csv-header">
+ <output-dir compare="Text">aws/s3/csv-header</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset" check-warnings="true">
+ <compilation-unit name="aws/s3/csv-no-header">
+ <output-dir compare="Text">aws/s3/csv-no-header</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset" check-warnings="true">
+ <compilation-unit name="aws/s3/tsv-header">
+ <output-dir compare="Text">aws/s3/tsv-header</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset" check-warnings="true">
+ <compilation-unit name="aws/s3/tsv-no-header">
+ <output-dir compare="Text">aws/s3/tsv-no-header</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset" check-warnings="true">
+ <compilation-unit name="aws/s3/csv-warnings">
+ <output-dir compare="Text">aws/s3/csv-warnings</output-dir>
+ <expected-warn>Parsing error at data_dir/no_h_missing_fields.csv line 2 field 3: some fields are missing</expected-warn>
+ <expected-warn>Parsing error at data_dir/no_h_no_closing_q.csv line 2 field 0: malformed input record ended abruptly</expected-warn>
+ <expected-warn>Parsing error at line 2 field 0: malformed input record ended abruptly</expected-warn>
+
+ <expected-warn>Parsing error at line 5 field 3: invalid value</expected-warn>
+ <expected-warn>Parsing error at line 2 field 1: invalid value</expected-warn>
+ <expected-warn>Parsing error at line 11 field 1: invalid value</expected-warn>
+ <expected-warn>Parsing error at line 3 field 1: invalid value</expected-warn>
+ <expected-warn>Parsing error at line 4 field 1: invalid value</expected-warn>
+ <expected-warn>Parsing error at line 7 field 7: invalid value</expected-warn>
+ <expected-warn>Parsing error at line 13 field 7: invalid value</expected-warn>
+ <expected-warn>Parsing error at line 12 field 3: invalid value</expected-warn>
+ <expected-warn>Parsing error at line 9 field 6: a quote should be in the beginning</expected-warn>
+
+ <expected-warn>Parsing error at data_dir/h_invalid_values.csv line 5 field 3: invalid value</expected-warn>
+ <expected-warn>Parsing error at data_dir/h_invalid_values.csv line 2 field 1: invalid value</expected-warn>
+ <expected-warn>Parsing error at data_dir/h_invalid_values.csv line 11 field 1: invalid value</expected-warn>
+ <expected-warn>Parsing error at data_dir/h_invalid_values.csv line 3 field 1: invalid value</expected-warn>
+ <expected-warn>Parsing error at data_dir/h_invalid_values.csv line 4 field 1: invalid value</expected-warn>
+ <expected-warn>Parsing error at data_dir/h_invalid_values.csv line 7 field 7: invalid value</expected-warn>
+ <expected-warn>Parsing error at data_dir/h_invalid_values.csv line 13 field 7: invalid value</expected-warn>
+ <expected-warn>Parsing error at data_dir/h_invalid_values.csv line 12 field 3: invalid value</expected-warn>
+ <expected-warn>Parsing error at data_dir/h_invalid_values.csv line 9 field 6: a quote should be in the beginning</expected-warn>
+
+ <expected-warn>Parsing error at data_dir/error1_line_num.csv line 3 field 2: a quote enclosing a field needs to be followed by the delimiter</expected-warn>
+ <expected-warn>Parsing error at data_dir/error2_line_num.csv line 4 field 2: a quote enclosing a field needs to be followed by the delimiter</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset" check-warnings="true">
+ <compilation-unit name="aws/s3/tsv-warnings">
+ <output-dir compare="Text">aws/s3/tsv-warnings</output-dir>
+ <expected-warn>Parsing error at data_dir/no_h_missing_fields.tsv line 2 field 3: some fields are missing</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset" check-warnings="true">
+ <compilation-unit name="aws/s3/json-warnings">
+ <output-dir compare="Text">aws/s3/json-warnings</output-dir>
+ <expected-warn>Parsing error at data_dir/1.json line 3 field 0: malformed input record ended abruptly</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="external-dataset" check-warnings="true">
+ <compilation-unit name="aws/s3/jsonl">
+ <output-dir compare="Text">aws/s3/jsonl</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
+</test-suite>
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 cc2cd24..600dde8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -31,15 +31,15 @@
<test-case FilePath="api">
<compilation-unit name="readonly-request">
<output-dir compare="Text">readonly-request</output-dir>
- <expected-error>ASX0044: CREATE_DATAVERSE statement is prohibited by this request</expected-error>
- <expected-error>ASX0044: DATAVERSE_DROP statement is prohibited by this request</expected-error>
- <expected-error>ASX0044: DATASET_DECL statement is prohibited by this request</expected-error>
- <expected-error>ASX0044: DATASET_DROP statement is prohibited by this request</expected-error>
- <expected-error>ASX0044: CREATE_INDEX statement is prohibited by this request</expected-error>
- <expected-error>ASX0044: INDEX_DROP statement is prohibited by this request</expected-error>
- <expected-error>ASX0044: INSERT statement is prohibited by this request</expected-error>
- <expected-error>ASX0044: UPSERT statement is prohibited by this request</expected-error>
- <expected-error>ASX0044: DELETE statement is prohibited by this request</expected-error>
+ <expected-error>ASX0044: CREATE_DATAVERSE statement is not supported in read-only mode</expected-error>
+ <expected-error>ASX0044: DATAVERSE_DROP statement is not supported in read-only mode</expected-error>
+ <expected-error>ASX0044: DATASET_DECL statement is not supported in read-only mode</expected-error>
+ <expected-error>ASX0044: DATASET_DROP statement is not supported in read-only mode</expected-error>
+ <expected-error>ASX0044: CREATE_INDEX statement is not supported in read-only mode</expected-error>
+ <expected-error>ASX0044: INDEX_DROP statement is not supported in read-only mode</expected-error>
+ <expected-error>ASX0044: INSERT statement is not supported in read-only mode</expected-error>
+ <expected-error>ASX0044: UPSERT statement is not supported in read-only mode</expected-error>
+ <expected-error>ASX0044: DELETE statement is not supported in read-only mode</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="api">
@@ -108,6 +108,30 @@
<output-dir compare="AST">format-param-in-accept-05</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="api">
+ <compilation-unit name="get-non-query">
+ <output-dir compare="Text">get-non-query</output-dir>
+ <expected-error>CREATE_DATAVERSE statement is not supported in read-only mode</expected-error>
+ <expected-error>TYPE_DECL statement is not supported in read-only mode</expected-error>
+ <expected-error>CREATE_FUNCTION statement is not supported in read-only mode</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="api">
+ <compilation-unit name="get-query">
+ <output-dir compare="Text">get-query</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="api">
+ <compilation-unit name="post-non-query">
+ <output-dir compare="Text">post-non-query</output-dir>
+ <expected-error>TYPE_DECL statement is not supported in read-only mode</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="api">
+ <compilation-unit name="ignore-body-for-get">
+ <output-dir compare="Text">ignore-body-for-get</output-dir>
+ </compilation-unit>
+ </test-case>
</test-group>
<test-group name="flwor">
<test-case FilePath="flwor">
@@ -303,10 +327,16 @@
<output-dir compare="Text">agg_number_rec</output-dir>
</compilation-unit>
</test-case>
- <test-case FilePath="aggregate">
+ <test-case FilePath="aggregate" check-warnings="true">
<compilation-unit name="avg_mixed">
<output-dir compare="Text">avg_mixed</output-dir>
- <expected-error>Type incompatibility: function agg-avg gets incompatible input values: string and float</expected-error>
+ <expected-warn>Unsupported type: agg-avg cannot process input type string (in line 26, at column 16)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate" check-warnings="true">
+ <compilation-unit name="serial_avg_mixed">
+ <output-dir compare="Text">serial_avg_mixed</output-dir>
+ <expected-warn>Unsupported type: agg-avg cannot process input type string (in line 29, at column 39)</expected-warn>
</compilation-unit>
</test-case>
<test-case FilePath="aggregate">
@@ -789,6 +819,71 @@
</compilation-unit>
</test-case>
<test-case FilePath="aggregate">
+ <compilation-unit name="serial_avg_empty">
+ <output-dir compare="Text">serial_avg_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_avg_int8">
+ <output-dir compare="Text">serial_avg_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_avg_int8_null">
+ <output-dir compare="Text">serial_avg_int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_avg_int16">
+ <output-dir compare="Text">serial_avg_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_avg_int16_null">
+ <output-dir compare="Text">serial_avg_int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_avg_int32">
+ <output-dir compare="Text">serial_avg_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_avg_int32_null">
+ <output-dir compare="Text">serial_avg_int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_avg_int64">
+ <output-dir compare="Text">serial_avg_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_avg_int64_null">
+ <output-dir compare="Text">serial_avg_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_avg_float">
+ <output-dir compare="Text">serial_avg_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_avg_float_null">
+ <output-dir compare="Text">serial_avg_float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_avg_double">
+ <output-dir compare="Text">serial_avg_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
+ <compilation-unit name="serial_avg_double_null">
+ <output-dir compare="Text">serial_avg_double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate">
<compilation-unit name="serial_kurtosis_double">
<output-dir compare="Text">serial_kurtosis_double</output-dir>
</compilation-unit>
@@ -1497,10 +1592,16 @@
<output-dir compare="Text">agg_number_rec</output-dir>
</compilation-unit>
</test-case>
- <test-case FilePath="aggregate-sql">
+ <test-case FilePath="aggregate-sql" check-warnings="true">
<compilation-unit name="avg_mixed">
<output-dir compare="Text">avg_mixed</output-dir>
- <expected-error>Type incompatibility: function agg-avg gets incompatible input values: string and float</expected-error>
+ <expected-warn>Unsupported type: agg-avg cannot process input type string (in line 26, at column 16)</expected-warn>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql" check-warnings="true">
+ <compilation-unit name="serial_avg_mixed">
+ <output-dir compare="Text">serial_avg_mixed</output-dir>
+ <expected-warn>Unsupported type: agg-avg cannot process input type string (in line 29, at column 38)</expected-warn>
</compilation-unit>
</test-case>
<test-case FilePath="aggregate-sql">
@@ -1981,6 +2082,71 @@
</compilation-unit>
</test-case>
<test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_avg_empty">
+ <output-dir compare="Text">serial_avg_empty</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_avg_int8">
+ <output-dir compare="Text">serial_avg_int8</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_avg_int8_null">
+ <output-dir compare="Text">serial_avg_int8_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_avg_int16">
+ <output-dir compare="Text">serial_avg_int16</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_avg_int16_null">
+ <output-dir compare="Text">serial_avg_int16_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_avg_int32">
+ <output-dir compare="Text">serial_avg_int32</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_avg_int32_null">
+ <output-dir compare="Text">serial_avg_int32_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_avg_int64">
+ <output-dir compare="Text">serial_avg_int64</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_avg_int64_null">
+ <output-dir compare="Text">serial_avg_int64_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_avg_float">
+ <output-dir compare="Text">serial_avg_float</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_avg_float_null">
+ <output-dir compare="Text">serial_avg_float_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_avg_double">
+ <output-dir compare="Text">serial_avg_double</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
+ <compilation-unit name="serial_avg_double_null">
+ <output-dir compare="Text">serial_avg_double_null</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="aggregate-sql">
<compilation-unit name="serial_kurtosis_double">
<output-dir compare="Text">serial_kurtosis_double</output-dir>
</compilation-unit>
@@ -3687,7 +3853,7 @@
</test-case>
-->
</test-group>
- <test-group name="ddl/create-index">
+ <test-group name="ddl">
<test-case FilePath="ddl/create-index">
<compilation-unit name="create-index-1">
<output-dir compare="Text">create-index-1</output-dir>
@@ -3718,11 +3884,111 @@
</compilation-unit>
</test-case>
<test-case FilePath="ddl">
+ <compilation-unit name="create-dataset-inline-type-1">
+ <output-dir compare="Text">create-dataset-inline-type-1</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="create-dataset-inline-type-2">
+ <output-dir compare="Text">create-dataset-inline-type-2</output-dir>
+ <expected-error>ASX1082: Cannot find datatype with name test.$d$t$i$Cust1</expected-error>
+ <expected-error>ASX1082: Cannot find datatype with name test.$d$t$i$Cust2</expected-error>
+ <expected-error>ASX1082: Cannot find datatype with name my_unknown_type</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
<compilation-unit name="drop-primary-index">
<output-dir compare="Text">drop-primary-index</output-dir>
<expected-error>Cannot drop index "ds". Drop dataset "ds" to remove this index</expected-error>
</compilation-unit>
</test-case>
+ <test-case FilePath="ddl" check-warnings="true">
+ <compilation-unit name="invalid-dataverse">
+ <output-dir compare="Text">invalid-dataverse</output-dir>
+ <source-location>false</source-location>
+ <expected-warn>Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-warn>
+ <expected-error>Cannot find dataverse with name fakeDataverse (in line 27, at column 1)</expected-error>
+ <expected-warn>Cannot find dataverse with name fakeDataverse (in line 29, at column 1)</expected-warn>
+ <expected-error>Cannot find dataverse with name fakeDataverse (in line 30, at column 1)</expected-error>
+ <expected-error>Cannot find datatype with name fakeDataverse.myType</expected-error>
+ <expected-error>Cannot find dataverse with name fakeDataverse (in line 30, at column 1)</expected-error>
+ <expected-error>Cannot find dataverse with name fakeDataverse (in line 32, at column 1)</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="invalid-dataverse-name">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1115: Invalid name for a database object: "<empty>"</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: " a"</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="invalid-dataset-name">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1115: Invalid name for a database object: "<empty>"</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: " a"</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="invalid-feed-name">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1115: Invalid name for a database object: "<empty>"</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: " a"</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="invalid-feed-policy-name">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1115: Invalid name for a database object: "<empty>"</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: " a"</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="invalid-index-name">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1115: Invalid name for a database object: "<empty>"</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: " a"</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="invalid-nodegroup-name">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1115: Invalid name for a database object: "<empty>"</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: " a"</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="invalid-type-name">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1115: Invalid name for a database object: "<empty>"</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: " a"</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="invalid-udf-name">
+ <output-dir compare="Text">none</output-dir>
+ <expected-error>ASX1115: Invalid name for a database object: "<empty>"</expected-error>
+ <expected-error>ASX1115: Invalid name for a database object: " a"</expected-error>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl">
+ <compilation-unit name="dataset-and-index-same-dataverse">
+ <output-dir compare="Text">dataset-and-index-same-dataverse</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="ddl" check-warnings="true">
+ <compilation-unit name="drop_dataset_invalid_dataverse">
+ <output-dir compare="Text">drop_dataset_invalid_dataverse</output-dir>
+ <expected-error>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-error>
+ <expected-error>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-error>
+ <expected-error>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-error>
+ <expected-error>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-error>
+ <expected-error>ASX1050: Cannot find dataset with name fakeDataset1 in dataverse realDataverse (in line 22, at column 1)</expected-error>
+ <expected-warn>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-warn>
+ <expected-warn>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-warn>
+ <expected-warn>ASX1063: Cannot find dataverse with name fakeDataverse (in line 22, at column 1)</expected-warn>
+ </compilation-unit>
+ </test-case>
</test-group>
<test-group name="dml">
<test-case FilePath="dml">
@@ -5052,6 +5318,21 @@
</compilation-unit>
</test-case>
<test-case FilePath="group-by">
+ <compilation-unit name="gby-case-01">
+ <output-dir compare="Text">gby-case-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="gby-cross-join">
+ <output-dir compare="Text">gby-cross-join</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
+ <compilation-unit name="gby-nested-01">
+ <output-dir compare="Text">gby-nested-01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="group-by">
<compilation-unit name="gby-record">
<output-dir compare="Text">gby-record</output-dir>
</compilation-unit>
@@ -5864,6 +6145,11 @@
<output-dir compare="Text">query-ASTERIXDB-159-3</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="list">
+ <compilation-unit name="var-in-list">
+ <output-dir compare="Text">var-in-list</output-dir>
+ </compilation-unit>
+ </test-case>
</test-group>
<test-group name="misc">
<test-case FilePath="misc">
@@ -6118,6 +6404,11 @@
</compilation-unit>
</test-case>
<test-case FilePath="misc">
+ <compilation-unit name="query-ASTERIXDB-2886">
+ <output-dir compare="Text">query-ASTERIXDB-2886</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="misc">
<compilation-unit name="unsupported_parameter">
<output-dir compare="Text">none</output-dir>
<expected-error>Query parameter compiler.joinmem is not supported</expected-error>
@@ -6438,6 +6729,11 @@
<output-dir compare="Text">btree-index-composite-key-03</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="open-index-non-enforced/index-selection">
+ <compilation-unit name="btree-index-composite-key-04">
+ <output-dir compare="Text">btree-index-composite-key-04</output-dir>
+ </compilation-unit>
+ </test-case>
<test-case FilePath="open-index-non-enforced/correlated-index-selection">
<compilation-unit name="btree-index-01">
<output-dir compare="Text">btree-index-01</output-dir>
@@ -8518,6 +8814,11 @@
</test-group>
<test-group name="statement-params">
<test-case FilePath="statement-params">
+ <compilation-unit name="index_01">
+ <output-dir compare="Text">index_01</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="statement-params">
<compilation-unit name="mixed_01">
<output-dir compare="Text">mixed_01</output-dir>
</compilation-unit>
@@ -8995,6 +9296,11 @@
</compilation-unit>
</test-case>
<test-case FilePath="string">
+ <compilation-unit name="substr-ASTERIXDB-2949">
+ <output-dir compare="Text">substr-ASTERIXDB-2949</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="string">
<compilation-unit name="substring-after-1">
<output-dir compare="Text">substring-after-1</output-dir>
</compilation-unit>
@@ -9229,6 +9535,11 @@
<output-dir compare="Text">query-ASTERIXDB-1674</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="subquery">
+ <compilation-unit name="query-ASTERIXDB-2815">
+ <output-dir compare="Text">query-ASTERIXDB-2815</output-dir>
+ </compilation-unit>
+ </test-case>
</test-group>
<test-group name="subset-collection">
<test-case FilePath="subset-collection">
@@ -11106,22 +11417,16 @@
<test-case FilePath="load">
<compilation-unit name="csv_05">
<output-dir compare="Text">csv_05</output-dir>
- <expected-error>At record: 1, field#: 4 - a quote enclosing a field needs to be placed in the beginning of that field</expected-error>
- <source-location>false</source-location>
</compilation-unit>
</test-case>
<test-case FilePath="load">
<compilation-unit name="csv_06">
<output-dir compare="Text">csv_06</output-dir>
- <expected-error>At record: 1, field#: 3 - a quote enclosing a field needs to be placed in the beginning of that field</expected-error>
- <source-location>false</source-location>
</compilation-unit>
</test-case>
<test-case FilePath="load">
<compilation-unit name="csv_07">
<output-dir compare="Text">csv_07</output-dir>
- <expected-error>At record: 1, field#: 3 - A quote enclosing a field needs to be followed by the delimiter</expected-error>
- <source-location>false</source-location>
</compilation-unit>
</test-case>
<test-case FilePath="load">
@@ -11567,6 +11872,16 @@
</compilation-unit>
</test-case>
<test-case FilePath="leftouterjoin">
+ <compilation-unit name="loj-02-push-select">
+ <output-dir compare="Text">loj-02-push-select</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="leftouterjoin">
+ <compilation-unit name="loj-03-no-listify">
+ <output-dir compare="Text">loj-03-no-listify</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="leftouterjoin">
<compilation-unit name="query_issue658">
<output-dir compare="Text">query_issue658</output-dir>
</compilation-unit>
@@ -11591,6 +11906,21 @@
<output-dir compare="Text">query_issue849-2</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="leftouterjoin">
+ <compilation-unit name="empty-dataset">
+ <output-dir compare="Text">empty-dataset</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="leftouterjoin">
+ <compilation-unit name="query-ASTERIXDB-2857">
+ <output-dir compare="Text">query-ASTERIXDB-2857</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="leftouterjoin">
+ <compilation-unit name="right_branch_opt_1">
+ <output-dir compare="Text">right_branch_opt_1</output-dir>
+ </compilation-unit>
+ </test-case>
</test-group>
<test-group name="index-leftouterjoin">
<test-case FilePath="index-leftouterjoin">
@@ -12095,6 +12425,18 @@
</compilation-unit>
</test-case>
</test-group>
+ <test-group name="csv-tsv-parser">
+ <test-case FilePath="csv-tsv-parser">
+ <compilation-unit name="csv-parser-001">
+ <output-dir compare="Text">csv-parser-001</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="csv-tsv-parser">
+ <compilation-unit name="tsv-parser-001">
+ <output-dir compare="Text">tsv-parser-001</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
<test-group name="binary">
<test-case FilePath="binary">
<compilation-unit name="parse">
@@ -12149,6 +12491,16 @@
</test-group>
<test-group name="unnest">
<test-case FilePath="unnest">
+ <compilation-unit name="ASTERIXDB-2750_unnest_join">
+ <output-dir compare="Text">ASTERIXDB-2750_unnest_join</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="unnest">
+ <compilation-unit name="ASTERIXDB-2844_unnest_syntax">
+ <output-dir compare="Text">ASTERIXDB-2844_unnest_syntax</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="unnest">
<compilation-unit name="left-outer-unnest">
<output-dir compare="Text">left-outer-unnest</output-dir>
</compilation-unit>
@@ -12183,6 +12535,11 @@
</compilation-unit>
</test-case>
<test-case FilePath="union">
+ <compilation-unit name="union_nested">
+ <output-dir compare="Text">union_nested</output-dir>
+ </compilation-unit>
+ </test-case>
+ <test-case FilePath="union">
<compilation-unit name="union_orderby">
<output-dir compare="Text">union_orderby</output-dir>
</compilation-unit>
diff --git a/asterixdb/asterix-common/pom.xml b/asterixdb/asterix-common/pom.xml
index 4e9aed7..d7704f1 100644
--- a/asterixdb/asterix-common/pom.xml
+++ b/asterixdb/asterix-common/pom.xml
@@ -38,114 +38,6 @@
<build>
<plugins>
<plugin>
- <groupId>org.jvnet.jaxb2.maven2</groupId>
- <artifactId>maven-jaxb2-plugin</artifactId>
- <executions>
- <execution>
- <id>configuration</id>
- <goals>
- <goal>generate</goal>
- </goals>
- <configuration>
- <args>
- <arg>-Xsetters</arg>
- <arg>-Xvalue-constructor</arg>
- </args>
- <plugins>
- <plugin>
- <groupId>org.jvnet.jaxb2_commons</groupId>
- <artifactId>jaxb2-basics</artifactId>
- <version>0.6.2</version>
- </plugin>
- <plugin>
- <groupId>org.jvnet.jaxb2_commons</groupId>
- <artifactId>jaxb2-value-constructor</artifactId>
- <version>3.0</version>
- </plugin>
- </plugins>
- <schemaDirectory>src/main/resources/schema</schemaDirectory>
- <schemaIncludes>
- <include>asterix-conf.xsd</include>
- </schemaIncludes>
- <generatePackage>org.apache.asterix.common.configuration</generatePackage>
- <bindingDirectory>src/main/resources/schema</bindingDirectory>
- <bindingIncludes>
- <bindingInclude>jaxb-bindings.xjb</bindingInclude>
- </bindingIncludes>
- <generateDirectory>${project.build.directory}/generated-sources/configuration</generateDirectory>
- </configuration>
- </execution>
- <execution>
- <id>cluster</id>
- <goals>
- <goal>generate</goal>
- </goals>
- <configuration>
- <args>
- <arg>-Xsetters</arg>
- <arg>-Xvalue-constructor</arg>
- </args>
- <plugins>
- <plugin>
- <groupId>org.jvnet.jaxb2_commons</groupId>
- <artifactId>jaxb2-basics</artifactId>
- <version>0.6.2</version>
- </plugin>
- <plugin>
- <groupId>org.jvnet.jaxb2_commons</groupId>
- <artifactId>jaxb2-value-constructor</artifactId>
- <version>3.0</version>
- </plugin>
- </plugins>
- <schemaDirectory>src/main/resources/schema</schemaDirectory>
- <schemaIncludes>
- <include>cluster.xsd</include>
- </schemaIncludes>
- <generatePackage>org.apache.asterix.event.schema.cluster</generatePackage>
- <generateDirectory>${project.build.directory}/generated-sources/cluster</generateDirectory>
- <bindingDirectory>src/main/resources/schema</bindingDirectory>
- <bindingIncludes>
- <bindingInclude>jaxb-bindings.xjb</bindingInclude>
- </bindingIncludes>
- </configuration>
- </execution>
- <execution>
- <id>yarn_cluster</id>
- <goals>
- <goal>generate</goal>
- </goals>
- <configuration>
- <args>
- <arg>-Xsetters</arg>
- <arg>-Xvalue-constructor</arg>
- </args>
- <plugins>
- <plugin>
- <groupId>org.jvnet.jaxb2_commons</groupId>
- <artifactId>jaxb2-basics</artifactId>
- <version>0.6.2</version>
- </plugin>
- <plugin>
- <groupId>org.jvnet.jaxb2_commons</groupId>
- <artifactId>jaxb2-value-constructor</artifactId>
- <version>3.0</version>
- </plugin>
- </plugins>
- <schemaDirectory>src/main/resources/schema</schemaDirectory>
- <schemaIncludes>
- <include>yarn_cluster.xsd</include>
- </schemaIncludes>
- <generatePackage>org.apache.asterix.event.schema.yarnCluster</generatePackage>
- <generateDirectory>${project.build.directory}/generated-sources/yarn_cluster</generateDirectory>
- <bindingDirectory>src/main/resources/schema</bindingDirectory>
- <bindingIncludes>
- <bindingInclude>jaxb-bindings.xjb</bindingInclude>
- </bindingIncludes>
- </configuration>
- </execution>
- </executions>
- </plugin>
- <plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-jar-plugin</artifactId>
<configuration>
@@ -247,7 +139,7 @@
</dependency>
<dependency>
<groupId>org.mockito</groupId>
- <artifactId>mockito-all</artifactId>
+ <artifactId>mockito-core</artifactId>
<scope>test</scope>
</dependency>
<dependency>
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ICoordinationService.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ICoordinationService.java
index 2019da9..5d2ef07 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ICoordinationService.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ICoordinationService.java
@@ -35,6 +35,18 @@
void put(String key, byte[] value) throws HyracksDataException;
/**
+ * Adds or updates the property with the name {@code key}
+ * with {@code value}, and indicates that values associated with
+ * this key should be treated as sensitive (e.g. not logged). Once a
+ * key is considered sensitive, it remains sensitive until deleted.
+ *
+ * @param key
+ * @param value
+ * @throws HyracksDataException
+ */
+ void putSensitive(String key, byte[] value) throws HyracksDataException;
+
+ /**
* Gets the value of the property with name {@code key} if exists.
*
* @param key
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetLifecycleManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetLifecycleManager.java
index 10eddb1..0aad368 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetLifecycleManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IDatasetLifecycleManager.java
@@ -19,6 +19,7 @@
package org.apache.asterix.common.api;
import java.util.List;
+import java.util.Set;
import java.util.function.Predicate;
import org.apache.asterix.common.context.DatasetInfo;
@@ -112,6 +113,14 @@
List<IVirtualBufferCache> getVirtualBufferCaches(int datasetId, int ioDeviceNum);
/**
+ * Attempts to close the datasets in {@code datasetsToClose}
+ *
+ * @param datasetsToClose
+ * @throws HyracksDataException
+ */
+ void closeDatasets(Set<Integer> datasetsToClose) throws HyracksDataException;
+
+ /**
* Flushes then closes all open datasets
*/
void closeAllDatasets() throws HyracksDataException;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IRequestReference.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IRequestReference.java
index 8a25ed2..c7f8d30 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IRequestReference.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IRequestReference.java
@@ -42,4 +42,18 @@
* @return the time at which the request was received.
*/
long getTime();
+
+ /**
+ * Gets the user agent from which the request was received.
+ *
+ * @return user agent from which the request was received.
+ */
+ String getUserAgent();
+
+ /**
+ * Gets the remote address from which the request was received.
+ *
+ * @return remote address from which the request was received.
+ */
+ String getRemoteAddr();
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
index 9428e6f..652f390 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
@@ -23,7 +23,6 @@
import static org.apache.hyracks.control.common.config.OptionTypes.INTEGER_BYTE_UNIT;
import static org.apache.hyracks.control.common.config.OptionTypes.LONG_BYTE_UNIT;
import static org.apache.hyracks.control.common.config.OptionTypes.POSITIVE_INTEGER;
-import static org.apache.hyracks.control.common.config.OptionTypes.UNSIGNED_INTEGER;
import static org.apache.hyracks.util.StorageUtil.StorageUnit.KILOBYTE;
import static org.apache.hyracks.util.StorageUtil.StorageUnit.MEGABYTE;
@@ -31,6 +30,7 @@
import org.apache.hyracks.api.config.IOption;
import org.apache.hyracks.api.config.IOptionType;
import org.apache.hyracks.api.config.Section;
+import org.apache.hyracks.control.common.config.OptionTypes;
import org.apache.hyracks.util.StorageUtil;
public class CompilerProperties extends AbstractProperties {
@@ -56,6 +56,10 @@
LONG_BYTE_UNIT,
StorageUtil.getLongSizeInBytes(32L, MEGABYTE),
"The memory budget (in bytes) for an inverted-index-search operator instance in a partition"),
+ COMPILER_EXTERNALSCANMEMORY(
+ INTEGER_BYTE_UNIT,
+ StorageUtil.getIntSizeInBytes(4, KILOBYTE),
+ "The memory budget (in bytes) for an external scan operator instance in a partition"),
COMPILER_FRAMESIZE(
INTEGER_BYTE_UNIT,
StorageUtil.getIntSizeInBytes(32, KILOBYTE),
@@ -68,7 +72,10 @@
+ "other integer values dictate the number of query execution parallel partitions. The system will "
+ "fall back to use the number of all available CPU cores in the cluster as the degree of parallelism "
+ "if the number set by a user is too large or too small"),
- COMPILER_STRINGOFFSET(UNSIGNED_INTEGER, 0, "Position of a first character in a String/Binary (0 or 1)"),
+ COMPILER_STRINGOFFSET(
+ OptionTypes.getRangedIntegerType(0, 1),
+ 0,
+ "Position of a first character in a String/Binary (0 or 1)"),
COMPILER_SORT_PARALLEL(BOOLEAN, AlgebricksConfig.SORT_PARALLEL, "Enabling/Disabling full parallel sort"),
COMPILER_SORT_SAMPLES(
POSITIVE_INTEGER,
@@ -107,7 +114,7 @@
@Override
public boolean hidden() {
- return this == COMPILER_STRINGOFFSET;
+ return this == COMPILER_STRINGOFFSET || this == COMPILER_EXTERNALSCANMEMORY;
}
}
@@ -127,6 +134,8 @@
public static final String COMPILER_SORT_SAMPLES_KEY = Option.COMPILER_SORT_SAMPLES.ini();
+ public static final String COMPILER_EXTERNALSCANMEMORY_KEY = Option.COMPILER_EXTERNALSCANMEMORY.ini();
+
public static final int COMPILER_PARALLELISM_AS_STORAGE = 0;
public CompilerProperties(PropertiesAccessor accessor) {
@@ -173,4 +182,8 @@
public int getSortSamples() {
return accessor.getInt(Option.COMPILER_SORT_SAMPLES);
}
+
+ public int getExternalScanMemorySize() {
+ return accessor.getInt(Option.COMPILER_EXTERNALSCANMEMORY);
+ }
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java
index 1533c9f..642cbd6 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java
@@ -19,9 +19,10 @@
package org.apache.asterix.common.config;
import static org.apache.hyracks.control.common.config.OptionTypes.LEVEL;
+import static org.apache.hyracks.control.common.config.OptionTypes.NONNEGATIVE_INTEGER;
import static org.apache.hyracks.control.common.config.OptionTypes.POSITIVE_INTEGER;
+import static org.apache.hyracks.control.common.config.OptionTypes.POSITIVE_INTEGER_BYTE_UNIT;
import static org.apache.hyracks.control.common.config.OptionTypes.STRING;
-import static org.apache.hyracks.control.common.config.OptionTypes.UNSIGNED_INTEGER;
import org.apache.hyracks.api.config.IOption;
import org.apache.hyracks.api.config.IOptionType;
@@ -32,11 +33,11 @@
public class ExternalProperties extends AbstractProperties {
public enum Option implements IOption {
- WEB_PORT(UNSIGNED_INTEGER, 19001, "The listen port of the legacy query interface"),
- WEB_QUERYINTERFACE_PORT(UNSIGNED_INTEGER, 19006, "The listen port of the query web interface"),
- API_PORT(UNSIGNED_INTEGER, 19002, "The listen port of the API server"),
- ACTIVE_PORT(UNSIGNED_INTEGER, 19003, "The listen port of the active server"),
- NC_API_PORT(UNSIGNED_INTEGER, 19004, "The listen port of the node controller API server"),
+ WEB_PORT(NONNEGATIVE_INTEGER, 19001, "The listen port of the legacy query interface"),
+ WEB_QUERYINTERFACE_PORT(NONNEGATIVE_INTEGER, 19006, "The listen port of the query web interface"),
+ API_PORT(NONNEGATIVE_INTEGER, 19002, "The listen port of the API server"),
+ ACTIVE_PORT(NONNEGATIVE_INTEGER, 19003, "The listen port of the active server"),
+ NC_API_PORT(NONNEGATIVE_INTEGER, 19004, "The listen port of the node controller API server"),
LOG_LEVEL(LEVEL, Level.WARN, "The logging level for master and slave processes"),
MAX_WAIT_ACTIVE_CLUSTER(
POSITIVE_INTEGER,
@@ -46,10 +47,10 @@
CC_JAVA_OPTS(STRING, "-Xmx1024m", "The JVM options passed to the cluster controller process by managix"),
NC_JAVA_OPTS(STRING, "-Xmx1024m", "The JVM options passed to the node controller process(es) by managix"),
MAX_WEB_REQUEST_SIZE(
- UNSIGNED_INTEGER,
+ POSITIVE_INTEGER_BYTE_UNIT,
StorageUtil.getIntSizeInBytes(50, StorageUtil.StorageUnit.MEGABYTE),
"The maximum accepted web request size in bytes"),
- REQUESTS_ARCHIVE_SIZE(UNSIGNED_INTEGER, 50, "The maximum number of archived requests to maintain");
+ REQUESTS_ARCHIVE_SIZE(NONNEGATIVE_INTEGER, 50, "The maximum number of archived requests to maintain");
private final IOptionType type;
private final Object defaultValue;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MetadataProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MetadataProperties.java
index 7a3e707..7d5ec42 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MetadataProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/MetadataProperties.java
@@ -18,9 +18,9 @@
*/
package org.apache.asterix.common.config;
+import static org.apache.hyracks.control.common.config.OptionTypes.NONNEGATIVE_INTEGER;
import static org.apache.hyracks.control.common.config.OptionTypes.POSITIVE_INTEGER;
import static org.apache.hyracks.control.common.config.OptionTypes.STRING;
-import static org.apache.hyracks.control.common.config.OptionTypes.UNSIGNED_INTEGER;
import java.util.List;
import java.util.Map;
@@ -37,8 +37,8 @@
public enum Option implements IOption {
METADATA_NODE(STRING, null),
METADATA_REGISTRATION_TIMEOUT_SECS(POSITIVE_INTEGER, 60),
- METADATA_LISTEN_PORT(UNSIGNED_INTEGER, 0),
- METADATA_CALLBACK_PORT(UNSIGNED_INTEGER, 0);
+ METADATA_LISTEN_PORT(NONNEGATIVE_INTEGER, 0),
+ METADATA_CALLBACK_PORT(NONNEGATIVE_INTEGER, 0);
private final IOptionType type;
private final Object defaultValue;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
index 23fdcac..22fb13b 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/OptimizationConfUtil.java
@@ -59,6 +59,9 @@
int textSearchFrameLimit = getTextSearchNumFrames(compilerProperties, querySpecificConfig, sourceLoc);
int sortNumSamples = getSortSamples(compilerProperties, querySpecificConfig, sourceLoc);
boolean fullParallelSort = getSortParallel(compilerProperties, querySpecificConfig);
+ int externalScanBufferSize = getExternalScanBufferSize(
+ (String) querySpecificConfig.get(CompilerProperties.COMPILER_EXTERNALSCANMEMORY_KEY),
+ compilerProperties.getExternalScanMemorySize(), sourceLoc);
PhysicalOptimizationConfig physOptConf = new PhysicalOptimizationConfig();
physOptConf.setFrameSize(frameSize);
@@ -69,10 +72,22 @@
physOptConf.setMaxFramesForTextSearch(textSearchFrameLimit);
physOptConf.setSortParallel(fullParallelSort);
physOptConf.setSortSamples(sortNumSamples);
+ physOptConf.setExternalScanBufferSize(externalScanBufferSize);
return physOptConf;
}
+ private static int getExternalScanBufferSize(String externalScanMemorySizeParameter,
+ int compilerExternalScanMemorySize, SourceLocation sourceLoc) throws AsterixException {
+ IOptionType<Integer> intByteParser = OptionTypes.INTEGER_BYTE_UNIT;
+ try {
+ return externalScanMemorySizeParameter != null ? intByteParser.parse(externalScanMemorySizeParameter)
+ : compilerExternalScanMemorySize;
+ } catch (IllegalArgumentException e) {
+ throw AsterixException.create(ErrorCode.COMPILATION_ERROR, sourceLoc, e.getMessage());
+ }
+ }
+
public static int getSortNumFrames(CompilerProperties compilerProperties, Map<String, Object> querySpecificConfig,
SourceLocation sourceLoc) throws AlgebricksException {
return getFrameLimit(CompilerProperties.COMPILER_SORTMEMORY_KEY,
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
index 6082f30..dd42936 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
@@ -21,9 +21,9 @@
import static org.apache.hyracks.control.common.config.OptionTypes.BOOLEAN;
import static org.apache.hyracks.control.common.config.OptionTypes.INTEGER_BYTE_UNIT;
import static org.apache.hyracks.control.common.config.OptionTypes.LONG;
+import static org.apache.hyracks.control.common.config.OptionTypes.NONNEGATIVE_INTEGER;
import static org.apache.hyracks.control.common.config.OptionTypes.POSITIVE_INTEGER;
import static org.apache.hyracks.control.common.config.OptionTypes.STRING;
-import static org.apache.hyracks.control.common.config.OptionTypes.UNSIGNED_INTEGER;
import java.util.concurrent.TimeUnit;
@@ -51,7 +51,7 @@
TimeUnit.SECONDS.toSeconds(30),
"The time in seconds to timeout waiting for master or replica to ack"),
REPLICATION_ENABLED(BOOLEAN, false, "Whether or not data replication is enabled"),
- REPLICATION_FACTOR(UNSIGNED_INTEGER, 2, "Number of replicas (backups) to maintain per master replica"),
+ REPLICATION_FACTOR(NONNEGATIVE_INTEGER, 2, "Number of replicas (backups) to maintain per master replica"),
REPLICATION_STRATEGY(STRING, "none", "Replication strategy to choose");
private final IOptionType type;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java
index 58bc828..0bea099 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java
@@ -21,9 +21,9 @@
import static org.apache.hyracks.control.common.config.OptionTypes.DOUBLE;
import static org.apache.hyracks.control.common.config.OptionTypes.INTEGER_BYTE_UNIT;
import static org.apache.hyracks.control.common.config.OptionTypes.LONG_BYTE_UNIT;
+import static org.apache.hyracks.control.common.config.OptionTypes.NONNEGATIVE_INTEGER;
import static org.apache.hyracks.control.common.config.OptionTypes.POSITIVE_INTEGER;
import static org.apache.hyracks.control.common.config.OptionTypes.STRING;
-import static org.apache.hyracks.control.common.config.OptionTypes.UNSIGNED_INTEGER;
import static org.apache.hyracks.util.StorageUtil.StorageUnit.KILOBYTE;
import static org.apache.hyracks.util.StorageUtil.StorageUnit.MEGABYTE;
@@ -43,13 +43,13 @@
STORAGE_BUFFERCACHE_PAGESIZE(INTEGER_BYTE_UNIT, StorageUtil.getIntSizeInBytes(128, KILOBYTE)),
// By default, uses 1/4 of the maximum heap size for read cache, i.e., disk buffer cache.
STORAGE_BUFFERCACHE_SIZE(LONG_BYTE_UNIT, Runtime.getRuntime().maxMemory() / 4),
- STORAGE_BUFFERCACHE_MAXOPENFILES(UNSIGNED_INTEGER, Integer.MAX_VALUE),
+ STORAGE_BUFFERCACHE_MAXOPENFILES(NONNEGATIVE_INTEGER, Integer.MAX_VALUE),
STORAGE_MEMORYCOMPONENT_GLOBALBUDGET(LONG_BYTE_UNIT, Runtime.getRuntime().maxMemory() / 4),
STORAGE_MEMORYCOMPONENT_PAGESIZE(INTEGER_BYTE_UNIT, StorageUtil.getIntSizeInBytes(128, KILOBYTE)),
STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS(POSITIVE_INTEGER, 2),
STORAGE_METADATA_MEMORYCOMPONENT_NUMPAGES(POSITIVE_INTEGER, 8),
STORAGE_LSM_BLOOMFILTER_FALSEPOSITIVERATE(DOUBLE, 0.01d),
- STORAGE_MAX_ACTIVE_WRITABLE_DATASETS(UNSIGNED_INTEGER, 8),
+ STORAGE_MAX_ACTIVE_WRITABLE_DATASETS(POSITIVE_INTEGER, 8),
STORAGE_COMPRESSION_BLOCK(STRING, "snappy"),
STORAGE_DISK_FORCE_BYTES(LONG_BYTE_UNIT, StorageUtil.getLongSizeInBytes(16, MEGABYTE)),
STORAGE_IO_SCHEDULER(STRING, "greedy");
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/TransactionProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/TransactionProperties.java
index 5ed069c..f813ac3 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/TransactionProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/TransactionProperties.java
@@ -21,8 +21,8 @@
import static org.apache.hyracks.control.common.config.OptionTypes.BOOLEAN;
import static org.apache.hyracks.control.common.config.OptionTypes.INTEGER_BYTE_UNIT;
import static org.apache.hyracks.control.common.config.OptionTypes.LONG_BYTE_UNIT;
+import static org.apache.hyracks.control.common.config.OptionTypes.NONNEGATIVE_INTEGER;
import static org.apache.hyracks.control.common.config.OptionTypes.POSITIVE_INTEGER;
-import static org.apache.hyracks.control.common.config.OptionTypes.UNSIGNED_INTEGER;
import static org.apache.hyracks.util.StorageUtil.StorageUnit.MEGABYTE;
import java.util.Map;
@@ -59,9 +59,9 @@
120,
"The frequency (in seconds) the checkpoint thread should check to see if a checkpoint should be "
+ "written"),
- TXN_LOG_CHECKPOINT_HISTORY(UNSIGNED_INTEGER, 2, "The number of checkpoints to keep in the transaction log"),
+ TXN_LOG_CHECKPOINT_HISTORY(NONNEGATIVE_INTEGER, 2, "The number of checkpoints to keep in the transaction log"),
TXN_LOCK_ESCALATIONTHRESHOLD(
- UNSIGNED_INTEGER,
+ NONNEGATIVE_INTEGER,
1000,
"The maximum number of entity locks to obtain before upgrading to a dataset lock"),
TXN_LOCK_SHRINKTIMER(
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
index d396d9b..fdf7822 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/DatasetLifecycleManager.java
@@ -27,6 +27,7 @@
import java.util.Collections;
import java.util.List;
import java.util.Map;
+import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.function.Predicate;
@@ -474,6 +475,16 @@
}
@Override
+ public synchronized void closeDatasets(Set<Integer> datasetsToClose) throws HyracksDataException {
+ ArrayList<DatasetResource> openDatasets = new ArrayList<>(datasets.values());
+ for (DatasetResource dsr : openDatasets) {
+ if (dsr.isOpen() && datasetsToClose.contains(dsr.getDatasetID())) {
+ closeDataset(dsr);
+ }
+ }
+ }
+
+ @Override
public synchronized void closeAllDatasets() throws HyracksDataException {
ArrayList<DatasetResource> openDatasets = new ArrayList<>(datasets.values());
for (DatasetResource dsr : openDatasets) {
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java
index 3389962..5fc1bb7 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/ICcApplicationContext.java
@@ -19,6 +19,7 @@
package org.apache.asterix.common.dataflow;
import org.apache.asterix.common.api.IApplicationContext;
+import org.apache.asterix.common.api.ICoordinationService;
import org.apache.asterix.common.api.IMetadataLockManager;
import org.apache.asterix.common.api.INodeJobTracker;
import org.apache.asterix.common.api.IRequestTracker;
@@ -26,6 +27,7 @@
import org.apache.asterix.common.cluster.IGlobalRecoveryManager;
import org.apache.asterix.common.config.ExtensionProperties;
import org.apache.asterix.common.context.IStorageComponentProvider;
+import org.apache.asterix.common.external.IAdapterFactoryService;
import org.apache.asterix.common.metadata.IMetadataBootstrap;
import org.apache.asterix.common.replication.INcLifecycleCoordinator;
import org.apache.asterix.common.storage.ICompressionManager;
@@ -127,4 +129,18 @@
* @return the request tracker.
*/
IRequestTracker getRequestTracker();
+
+ /**
+ * Gets the coordination service
+ *
+ * @return the coordination service
+ */
+ ICoordinationService getCoordinationService();
+
+ /**
+ * Gets the adapter factory service
+ *
+ * @return the adapter factory service
+ */
+ IAdapterFactoryService getAdapterFactoryService();
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMInsertDeleteOperatorNodePushable.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMInsertDeleteOperatorNodePushable.java
index 5e4b5a1..128a8c0 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMInsertDeleteOperatorNodePushable.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/LSMInsertDeleteOperatorNodePushable.java
@@ -147,7 +147,7 @@
}
}
} catch (HyracksDataException e) {
- if (e.getErrorCode() == ErrorCode.INVALID_OPERATOR_OPERATION) {
+ if (e.matches(ErrorCode.INVALID_OPERATOR_OPERATION)) {
throw e;
} else {
throw HyracksDataException.create(ErrorCode.ERROR_PROCESSING_TUPLE, e, sourceLoc, i);
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/AsterixException.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/AsterixException.java
index a834a06..92714b3 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/AsterixException.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/AsterixException.java
@@ -26,54 +26,51 @@
public class AsterixException extends AlgebricksException {
private static final long serialVersionUID = 1L;
+ public AsterixException(ErrorCode error, Throwable cause, SourceLocation sourceLoc, Serializable... params) {
+ super(error, cause, sourceLoc, params);
+ }
+
+ public AsterixException(ErrorCode error, SourceLocation sourceLoc, Serializable... params) {
+ this(error, null, sourceLoc, params);
+ }
+
+ public AsterixException(ErrorCode error, Serializable... params) {
+ super(error, null, null, params);
+ }
+
+ public AsterixException(ErrorCode error, Throwable cause, Serializable... params) {
+ super(error, cause, null, params);
+ }
+
/**
- * @deprecated Instead, use a constructor with error code
+ * @deprecated Instead, use a constructor with {@link ErrorCode}
*/
@Deprecated
public AsterixException(String message) {
super(message);
}
- public AsterixException(int errorCode, SourceLocation sourceLoc, Serializable... params) {
- super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), sourceLoc, params);
- }
-
- public AsterixException(int errorCode, Serializable... params) {
- super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), params);
- }
-
/**
- * @deprecated When creating a constructor with cause,
- * create AlgebricksException using AlgebricksException.create(Throwable th);
+ * @deprecated Instead, use a constructor with {@link ErrorCode}
*/
@Deprecated
public AsterixException(Throwable cause) {
super(cause);
}
- public AsterixException(int errorCode, Throwable cause, SourceLocation sourceLoc, Serializable... params) {
- super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), sourceLoc, params);
- addSuppressed(cause);
- }
-
- public AsterixException(int errorCode, Throwable cause, Serializable... params) {
- super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), params);
- addSuppressed(cause);
- }
-
/**
- * @deprecated Instead, use a constructor with error code
+ * @deprecated Instead, use a constructor with {@link ErrorCode}
*/
@Deprecated
public AsterixException(String message, Throwable cause) {
super(message, cause);
}
- public static AsterixException create(int errorCode, SourceLocation sourceLoc, Serializable... params) {
- return new AsterixException(errorCode, sourceLoc, params);
+ public static AsterixException create(ErrorCode error, SourceLocation sourceLoc, Serializable... params) {
+ return new AsterixException(error, sourceLoc, params);
}
- public static AsterixException create(int errorCode, Serializable... params) {
- return new AsterixException(errorCode, params);
+ public static AsterixException create(ErrorCode error, Serializable... params) {
+ return new AsterixException(error, params);
}
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/CompilationException.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/CompilationException.java
index 0de6b72..75fb18d 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/CompilationException.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/CompilationException.java
@@ -27,20 +27,28 @@
public class CompilationException extends AlgebricksException {
private static final long serialVersionUID = 1L;
- public CompilationException(int errorCode, SourceLocation sourceLoc, Serializable... params) {
- super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), sourceLoc, params);
+ public static CompilationException create(ErrorCode error, SourceLocation sourceLoc, Serializable... params) {
+ return new CompilationException(error, sourceLoc, params);
}
- public CompilationException(int errorCode, Serializable... params) {
- super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), params);
+ public static CompilationException create(ErrorCode error, Serializable... params) {
+ return create(error, null, params);
}
- public CompilationException(int errorCode, Throwable cause, SourceLocation sourceLoc, Serializable... params) {
- super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), cause, sourceLoc, params);
+ public CompilationException(ErrorCode error, Throwable cause, SourceLocation sourceLoc, Serializable... params) {
+ super(error, cause, sourceLoc, params);
}
- public CompilationException(int errorCode, Throwable cause, Serializable... params) {
- super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), cause, params);
+ public CompilationException(ErrorCode error, SourceLocation sourceLoc, Serializable... params) {
+ this(error, null, sourceLoc, params);
+ }
+
+ public CompilationException(ErrorCode error, Serializable... params) {
+ this(error, null, null, params);
+ }
+
+ public CompilationException(ErrorCode errorCode, Throwable cause, Serializable... params) {
+ this(errorCode, cause, null, params);
}
/**
@@ -56,7 +64,7 @@
/**
* @deprecated (Don't use this and provide an error code. This exists for the current exceptions and
* those exceptions need to adopt error code as well.)
- * @param message
+ * @param cause
*/
@Deprecated
public CompilationException(Throwable cause) {
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index da1c4e7..192113e 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -18,9 +18,7 @@
*/
package org.apache.asterix.common.exceptions;
-import java.io.InputStream;
-import java.util.Map;
-
+import org.apache.hyracks.api.exceptions.IError;
import org.apache.hyracks.api.util.ErrorMessageUtil;
// Error code:
@@ -29,322 +27,337 @@
// 2000 ---- 2999: storage errors
// 3000 ---- 3999: feed errors
// 4000 ---- 4999: lifecycle management errors
-public class ErrorCode {
- private static final String RESOURCE_PATH = "asx_errormsg/en.properties";
- public static final String ASTERIX = "ASX";
-
+public enum ErrorCode implements IError {
// Runtime errors
- public static final int CASTING_FIELD = 1;
- public static final int TYPE_MISMATCH_FUNCTION = 2;
- public static final int TYPE_INCOMPATIBLE = 3;
- public static final int TYPE_UNSUPPORTED = 4;
- public static final int TYPE_ITEM = 5;
- public static final int INVALID_FORMAT = 6;
- public static final int OVERFLOW = 7;
- public static final int UNDERFLOW = 8;
- public static final int INJECTED_FAILURE = 9;
- public static final int NEGATIVE_VALUE = 10;
- public static final int OUT_OF_BOUND = 11;
- public static final int COERCION = 12;
- public static final int DUPLICATE_FIELD_NAME = 13;
- public static final int PROPERTY_NOT_SET = 14;
- public static final int ROOT_LOCAL_RESOURCE_EXISTS = 15;
- public static final int ROOT_LOCAL_RESOURCE_COULD_NOT_BE_CREATED = 16;
- public static final int UNKNOWN_EXTERNAL_FILE_PENDING_OP = 17;
- public static final int TYPE_CONVERT = 18;
- public static final int TYPE_CONVERT_INTEGER_SOURCE = 19;
- public static final int TYPE_CONVERT_INTEGER_TARGET = 20;
- public static final int TYPE_CONVERT_OUT_OF_BOUND = 21;
- public static final int FIELD_SHOULD_BE_TYPED = 22;
- public static final int NC_REQUEST_TIMEOUT = 23;
- public static final int POLYGON_INVALID_COORDINATE = 24;
- public static final int POLYGON_3_POINTS = 25;
- public static final int POLYGON_INVALID = 26;
- public static final int OPERATION_NOT_SUPPORTED = 27;
- public static final int INVALID_DURATION = 28;
- public static final int UNKNOWN_DURATION_UNIT = 29;
- public static final int REQUEST_TIMEOUT = 30;
- public static final int INVALID_TYPE_CASTING_MATH_FUNCTION = 31;
- public static final int REJECT_BAD_CLUSTER_STATE = 32;
- public static final int REJECT_NODE_UNREGISTERED = 33;
- public static final int UNSUPPORTED_MULTIPLE_STATEMENTS = 35;
- public static final int CANNOT_COMPARE_COMPLEX = 36;
- public static final int TYPE_MISMATCH_GENERIC = 37;
- public static final int DIFFERENT_LIST_TYPE_ARGS = 38;
- public static final int INTEGER_VALUE_EXPECTED = 39;
- public static final int NO_STATEMENT_PROVIDED = 40;
- public static final int REQUEST_CANCELLED = 41;
- public static final int TPCDS_INVALID_TABLE_NAME = 42;
- public static final int VALUE_OUT_OF_RANGE = 43;
- public static final int PROHIBITED_STATEMENT_CATEGORY = 44;
- public static final int INTEGER_VALUE_EXPECTED_FUNCTION = 45;
- public static final int INVALID_LIKE_PATTERN = 46;
- public static final int INVALID_REQ_PARAM_VAL = 47;
- public static final int INVALID_REQ_JSON_VAL = 48;
+ CASTING_FIELD(1),
+ TYPE_MISMATCH_FUNCTION(2),
+ TYPE_INCOMPATIBLE(3),
+ TYPE_UNSUPPORTED(4),
+ TYPE_ITEM(5),
+ INVALID_FORMAT(6),
+ OVERFLOW(7),
+ UNDERFLOW(8),
+ INJECTED_FAILURE(9),
+ NEGATIVE_VALUE(10),
+ OUT_OF_BOUND(11),
+ COERCION(12),
+ DUPLICATE_FIELD_NAME(13),
+ PROPERTY_NOT_SET(14),
+ ROOT_LOCAL_RESOURCE_EXISTS(15),
+ ROOT_LOCAL_RESOURCE_COULD_NOT_BE_CREATED(16),
+ UNKNOWN_EXTERNAL_FILE_PENDING_OP(17),
+ TYPE_CONVERT(18),
+ TYPE_CONVERT_INTEGER_SOURCE(19),
+ TYPE_CONVERT_INTEGER_TARGET(20),
+ TYPE_CONVERT_OUT_OF_BOUND(21),
+ FIELD_SHOULD_BE_TYPED(22),
+ NC_REQUEST_TIMEOUT(23),
+ POLYGON_INVALID_COORDINATE(24),
+ POLYGON_3_POINTS(25),
+ POLYGON_INVALID(26),
+ OPERATION_NOT_SUPPORTED(27),
+ INVALID_DURATION(28),
+ UNKNOWN_DURATION_UNIT(29),
+ REQUEST_TIMEOUT(30),
+ INVALID_TYPE_CASTING_MATH_FUNCTION(31),
+ REJECT_BAD_CLUSTER_STATE(32),
+ REJECT_NODE_UNREGISTERED(33),
+ UNSUPPORTED_MULTIPLE_STATEMENTS(35),
+ CANNOT_COMPARE_COMPLEX(36),
+ TYPE_MISMATCH_GENERIC(37),
+ DIFFERENT_LIST_TYPE_ARGS(38),
+ INTEGER_VALUE_EXPECTED(39),
+ NO_STATEMENT_PROVIDED(40),
+ REQUEST_CANCELLED(41),
+ TPCDS_INVALID_TABLE_NAME(42),
+ VALUE_OUT_OF_RANGE(43),
+ PROHIBITED_STATEMENT_CATEGORY(44),
+ INTEGER_VALUE_EXPECTED_FUNCTION(45),
+ INVALID_LIKE_PATTERN(46),
+ INVALID_REQ_PARAM_VAL(47),
+ INVALID_REQ_JSON_VAL(48),
+ PARAMETERS_REQUIRED(49),
+ INVALID_PARAM(50),
- public static final int UNSUPPORTED_JRE = 100;
+ UNSUPPORTED_JRE(100),
- public static final int EXTERNAL_UDF_RESULT_TYPE_ERROR = 200;
+ EXTERNAL_UDF_RESULT_TYPE_ERROR(200),
// Compilation errors
- public static final int PARSE_ERROR = 1001;
- public static final int COMPILATION_TYPE_MISMATCH_FUNCTION = 1002;
- public static final int COMPILATION_TYPE_INCOMPATIBLE = 1003;
- public static final int COMPILATION_TYPE_UNSUPPORTED = 1004;
- public static final int COMPILATION_TYPE_ITEM = 1005;
- public static final int COMPILATION_DUPLICATE_FIELD_NAME = 1006;
- public static final int COMPILATION_INVALID_EXPRESSION = 1007;
- public static final int COMPILATION_INVALID_PARAMETER_NUMBER = 1008;
- public static final int COMPILATION_INVALID_RETURNING_EXPRESSION = 1009;
- public static final int COMPILATION_FULLTEXT_PHRASE_FOUND = 1010;
- public static final int COMPILATION_UNKNOWN_DATASET_TYPE = 1011;
- public static final int COMPILATION_UNKNOWN_INDEX_TYPE = 1012;
- public static final int COMPILATION_ILLEGAL_INDEX_NUM_OF_FIELD = 1013;
- public static final int COMPILATION_FIELD_NOT_FOUND = 1014;
- public static final int COMPILATION_ILLEGAL_INDEX_FOR_DATASET_WITH_COMPOSITE_PRIMARY_INDEX = 1015;
- public static final int COMPILATION_INDEX_TYPE_NOT_SUPPORTED_FOR_DATASET_TYPE = 1016;
- public static final int COMPILATION_FILTER_CANNOT_BE_NULLABLE = 1017;
- public static final int COMPILATION_ILLEGAL_FILTER_TYPE = 1018;
- public static final int COMPILATION_CANNOT_AUTOGENERATE_COMPOSITE_PRIMARY_KEY = 1019;
- public static final int COMPILATION_ILLEGAL_AUTOGENERATED_TYPE = 1020;
- public static final int COMPILATION_PRIMARY_KEY_CANNOT_BE_NULLABLE = 1021;
- public static final int COMPILATION_ILLEGAL_PRIMARY_KEY_TYPE = 1022;
- public static final int COMPILATION_CANT_DROP_ACTIVE_DATASET = 1023;
- public static final int COMPILATION_AQLPLUS_IDENTIFIER_NOT_FOUND = 1024;
- public static final int COMPILATION_AQLPLUS_NO_SUCH_JOIN_TYPE = 1025;
- public static final int COMPILATION_FUNC_EXPRESSION_CANNOT_UTILIZE_INDEX = 1026;
- public static final int COMPILATION_DATASET_TYPE_DOES_NOT_HAVE_PRIMARY_INDEX = 1027;
- public static final int COMPILATION_UNSUPPORTED_QUERY_PARAMETER = 1028;
- public static final int NO_METADATA_FOR_DATASET = 1029;
- public static final int SUBTREE_HAS_NO_DATA_SOURCE = 1030;
- public static final int SUBTREE_HAS_NO_ADDTIONAL_DATA_SOURCE = 1031;
- public static final int NO_INDEX_FIELD_NAME_FOR_GIVEN_FUNC_EXPR = 1032;
- public static final int NO_SUPPORTED_TYPE = 1033;
- public static final int NO_TOKENIZER_FOR_TYPE = 1034;
- public static final int INCOMPATIBLE_SEARCH_MODIFIER = 1035;
- public static final int UNKNOWN_SEARCH_MODIFIER = 1036;
- public static final int COMPILATION_BAD_QUERY_PARAMETER_VALUE = 1037;
- public static final int COMPILATION_ILLEGAL_STATE = 1038;
- public static final int COMPILATION_TWO_PHASE_LOCKING_VIOLATION = 1039;
- public static final int DATASET_ID_EXHAUSTED = 1040;
- public static final int INDEX_ILLEGAL_ENFORCED_NON_OPTIONAL = 1041;
- public static final int INDEX_ILLEGAL_NON_ENFORCED_TYPED = 1042;
- public static final int INDEX_RTREE_MULTIPLE_FIELDS_NOT_ALLOWED = 1043;
- public static final int REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE = 1044;
- public static final int ILLEGAL_LOCK_UPGRADE_OPERATION = 1045;
- public static final int ILLEGAL_LOCK_DOWNGRADE_OPERATION = 1046;
- public static final int UPGRADE_FAILED_LOCK_WAS_NOT_ACQUIRED = 1047;
- public static final int DOWNGRADE_FAILED_LOCK_WAS_NOT_ACQUIRED = 1048;
- public static final int LOCK_WAS_ACQUIRED_DIFFERENT_OPERATION = 1049;
- public static final int UNKNOWN_DATASET_IN_DATAVERSE = 1050;
- public static final int INDEX_ILLEGAL_ENFORCED_ON_CLOSED_FIELD = 1051;
- public static final int INDEX_ILLEGAL_REPETITIVE_FIELD = 1052;
- public static final int CANNOT_CREATE_SEC_PRIMARY_IDX_ON_EXT_DATASET = 1053;
- public static final int COMPILATION_FAILED_DUE_TO_REPLICATE_OP = 1054;
- public static final int COMPILATION_INCOMPATIBLE_FUNCTION_LANGUAGE = 1055;
- public static final int TOO_MANY_OPTIONS_FOR_FUNCTION = 1056;
- public static final int EXPRESSION_NOT_SUPPORTED_IN_CONSTANT_RECORD = 1057;
- public static final int LITERAL_TYPE_NOT_SUPPORTED_IN_CONSTANT_RECORD = 1058;
- public static final int UNSUPPORTED_WITH_FIELD = 1059;
- public static final int WITH_FIELD_MUST_BE_OF_TYPE = 1060;
- public static final int WITH_FIELD_MUST_CONTAIN_SUB_FIELD = 1061;
- public static final int CONFIGURATION_PARAMETER_INVALID_TYPE = 1062;
- public static final int UNKNOWN_DATAVERSE = 1063;
- public static final int ERROR_OCCURRED_BETWEEN_TWO_TYPES_CONVERSION = 1064;
- public static final int CHOSEN_INDEX_COUNT_SHOULD_BE_GREATER_THAN_ONE = 1065;
- public static final int CANNOT_SERIALIZE_A_VALUE = 1066;
- public static final int CANNOT_FIND_NON_MISSING_SELECT_OPERATOR = 1067;
- public static final int CANNOT_GET_CONDITIONAL_SPLIT_KEY_VARIABLE = 1068;
- public static final int CANNOT_DROP_INDEX = 1069;
- public static final int METADATA_ERROR = 1070;
- public static final int DATAVERSE_EXISTS = 1071;
- public static final int DATASET_EXISTS = 1072;
- public static final int UNDEFINED_IDENTIFIER = 1073;
- public static final int AMBIGUOUS_IDENTIFIER = 1074;
- public static final int FORBIDDEN_SCOPE = 1075;
- public static final int NAME_RESOLVE_UNKNOWN_DATASET = 1076;
- public static final int NAME_RESOLVE_UNKNOWN_DATASET_IN_DATAVERSE = 1077;
- public static final int COMPILATION_UNEXPECTED_OPERATOR = 1078;
- public static final int COMPILATION_ERROR = 1079;
- public static final int UNKNOWN_NODEGROUP = 1080;
- public static final int UNKNOWN_FUNCTION = 1081;
- public static final int UNKNOWN_TYPE = 1082;
- public static final int UNKNOWN_INDEX = 1083;
- public static final int INDEX_EXISTS = 1084;
- public static final int TYPE_EXISTS = 1085;
- public static final int PARAMETER_NO_VALUE = 1086;
- public static final int COMPILATION_INVALID_NUM_OF_ARGS = 1087;
- public static final int FIELD_NOT_FOUND = 1088;
- public static final int FIELD_NOT_OF_TYPE = 1089;
- public static final int ARRAY_FIELD_ELEMENTS_MUST_BE_OF_TYPE = 1090;
- public static final int COMPILATION_TYPE_MISMATCH_GENERIC = 1091;
- public static final int ILLEGAL_SET_PARAMETER = 1092;
- public static final int COMPILATION_TRANSLATION_ERROR = 1093;
- public static final int RANGE_MAP_ERROR = 1094;
- public static final int COMPILATION_EXPECTED_FUNCTION_CALL = 1095;
- public static final int UNKNOWN_COMPRESSION_SCHEME = 1096;
- public static final int UNSUPPORTED_WITH_SUBFIELD = 1097;
- public static final int COMPILATION_INVALID_WINDOW_FRAME = 1098;
- public static final int COMPILATION_UNEXPECTED_WINDOW_FRAME = 1099;
- public static final int COMPILATION_UNEXPECTED_WINDOW_EXPRESSION = 1100;
- public static final int COMPILATION_UNEXPECTED_WINDOW_ORDERBY = 1101;
- public static final int COMPILATION_EXPECTED_WINDOW_FUNCTION = 1102;
- public static final int COMPILATION_ILLEGAL_USE_OF_IDENTIFIER = 1103;
- public static final int INVALID_FUNCTION_MODIFIER = 1104;
- public static final int OPERATION_NOT_SUPPORTED_ON_PRIMARY_INDEX = 1105;
- public static final int EXPECTED_CONSTANT_VALUE = 1106;
- public static final int UNEXPECTED_HINT = 1107;
+ PARSE_ERROR(1001),
+ COMPILATION_TYPE_MISMATCH_FUNCTION(1002),
+ COMPILATION_TYPE_INCOMPATIBLE(1003),
+ COMPILATION_TYPE_UNSUPPORTED(1004),
+ COMPILATION_TYPE_ITEM(1005),
+ COMPILATION_DUPLICATE_FIELD_NAME(1006),
+ COMPILATION_INVALID_EXPRESSION(1007),
+ COMPILATION_INVALID_PARAMETER_NUMBER(1008),
+ COMPILATION_INVALID_RETURNING_EXPRESSION(1009),
+ COMPILATION_FULLTEXT_PHRASE_FOUND(1010),
+ COMPILATION_UNKNOWN_DATASET_TYPE(1011),
+ COMPILATION_UNKNOWN_INDEX_TYPE(1012),
+ COMPILATION_ILLEGAL_INDEX_NUM_OF_FIELD(1013),
+ COMPILATION_FIELD_NOT_FOUND(1014),
+ COMPILATION_ILLEGAL_INDEX_FOR_DATASET_WITH_COMPOSITE_PRIMARY_INDEX(1015),
+ COMPILATION_INDEX_TYPE_NOT_SUPPORTED_FOR_DATASET_TYPE(1016),
+ COMPILATION_FILTER_CANNOT_BE_NULLABLE(1017),
+ COMPILATION_ILLEGAL_FILTER_TYPE(1018),
+ COMPILATION_CANNOT_AUTOGENERATE_COMPOSITE_PRIMARY_KEY(1019),
+ COMPILATION_ILLEGAL_AUTOGENERATED_TYPE(1020),
+ COMPILATION_PRIMARY_KEY_CANNOT_BE_NULLABLE(1021),
+ COMPILATION_ILLEGAL_PRIMARY_KEY_TYPE(1022),
+ COMPILATION_CANT_DROP_ACTIVE_DATASET(1023),
+ COMPILATION_AQLPLUS_IDENTIFIER_NOT_FOUND(1024),
+ COMPILATION_AQLPLUS_NO_SUCH_JOIN_TYPE(1025),
+ COMPILATION_FUNC_EXPRESSION_CANNOT_UTILIZE_INDEX(1026),
+ COMPILATION_DATASET_TYPE_DOES_NOT_HAVE_PRIMARY_INDEX(1027),
+ COMPILATION_UNSUPPORTED_QUERY_PARAMETER(1028),
+ NO_METADATA_FOR_DATASET(1029),
+ SUBTREE_HAS_NO_DATA_SOURCE(1030),
+ SUBTREE_HAS_NO_ADDTIONAL_DATA_SOURCE(1031),
+ NO_INDEX_FIELD_NAME_FOR_GIVEN_FUNC_EXPR(1032),
+ NO_SUPPORTED_TYPE(1033),
+ NO_TOKENIZER_FOR_TYPE(1034),
+ INCOMPATIBLE_SEARCH_MODIFIER(1035),
+ UNKNOWN_SEARCH_MODIFIER(1036),
+ COMPILATION_BAD_QUERY_PARAMETER_VALUE(1037),
+ COMPILATION_ILLEGAL_STATE(1038),
+ COMPILATION_TWO_PHASE_LOCKING_VIOLATION(1039),
+ DATASET_ID_EXHAUSTED(1040),
+ INDEX_ILLEGAL_ENFORCED_NON_OPTIONAL(1041),
+ INDEX_ILLEGAL_NON_ENFORCED_TYPED(1042),
+ INDEX_RTREE_MULTIPLE_FIELDS_NOT_ALLOWED(1043),
+ REMOTE_EXCEPTION_WHEN_CALLING_METADATA_NODE(1044),
+ ILLEGAL_LOCK_UPGRADE_OPERATION(1045),
+ ILLEGAL_LOCK_DOWNGRADE_OPERATION(1046),
+ UPGRADE_FAILED_LOCK_WAS_NOT_ACQUIRED(1047),
+ DOWNGRADE_FAILED_LOCK_WAS_NOT_ACQUIRED(1048),
+ LOCK_WAS_ACQUIRED_DIFFERENT_OPERATION(1049),
+ UNKNOWN_DATASET_IN_DATAVERSE(1050),
+ INDEX_ILLEGAL_ENFORCED_ON_CLOSED_FIELD(1051),
+ INDEX_ILLEGAL_REPETITIVE_FIELD(1052),
+ CANNOT_CREATE_SEC_PRIMARY_IDX_ON_EXT_DATASET(1053),
+ COMPILATION_FAILED_DUE_TO_REPLICATE_OP(1054),
+ COMPILATION_INCOMPATIBLE_FUNCTION_LANGUAGE(1055),
+ TOO_MANY_OPTIONS_FOR_FUNCTION(1056),
+ EXPRESSION_NOT_SUPPORTED_IN_CONSTANT_RECORD(1057),
+ LITERAL_TYPE_NOT_SUPPORTED_IN_CONSTANT_RECORD(1058),
+ UNSUPPORTED_WITH_FIELD(1059),
+ WITH_FIELD_MUST_BE_OF_TYPE(1060),
+ WITH_FIELD_MUST_CONTAIN_SUB_FIELD(1061),
+ CONFIGURATION_PARAMETER_INVALID_TYPE(1062),
+ UNKNOWN_DATAVERSE(1063),
+ ERROR_OCCURRED_BETWEEN_TWO_TYPES_CONVERSION(1064),
+ CHOSEN_INDEX_COUNT_SHOULD_BE_GREATER_THAN_ONE(1065),
+ CANNOT_SERIALIZE_A_VALUE(1066),
+ CANNOT_FIND_NON_MISSING_SELECT_OPERATOR(1067),
+ CANNOT_GET_CONDITIONAL_SPLIT_KEY_VARIABLE(1068),
+ CANNOT_DROP_INDEX(1069),
+ METADATA_ERROR(1070),
+ DATAVERSE_EXISTS(1071),
+ DATASET_EXISTS(1072),
+ UNDEFINED_IDENTIFIER(1073),
+ AMBIGUOUS_IDENTIFIER(1074),
+ FORBIDDEN_SCOPE(1075),
+ NAME_RESOLVE_UNKNOWN_DATASET(1076),
+ NAME_RESOLVE_UNKNOWN_DATASET_IN_DATAVERSE(1077),
+ COMPILATION_UNEXPECTED_OPERATOR(1078),
+ COMPILATION_ERROR(1079),
+ UNKNOWN_NODEGROUP(1080),
+ UNKNOWN_FUNCTION(1081),
+ UNKNOWN_TYPE(1082),
+ UNKNOWN_INDEX(1083),
+ INDEX_EXISTS(1084),
+ TYPE_EXISTS(1085),
+ PARAMETER_NO_VALUE(1086),
+ COMPILATION_INVALID_NUM_OF_ARGS(1087),
+ FIELD_NOT_FOUND(1088),
+ FIELD_NOT_OF_TYPE(1089),
+ ARRAY_FIELD_ELEMENTS_MUST_BE_OF_TYPE(1090),
+ COMPILATION_TYPE_MISMATCH_GENERIC(1091),
+ ILLEGAL_SET_PARAMETER(1092),
+ COMPILATION_TRANSLATION_ERROR(1093),
+ RANGE_MAP_ERROR(1094),
+ COMPILATION_EXPECTED_FUNCTION_CALL(1095),
+ UNKNOWN_COMPRESSION_SCHEME(1096),
+ UNSUPPORTED_WITH_SUBFIELD(1097),
+ COMPILATION_INVALID_WINDOW_FRAME(1098),
+ COMPILATION_UNEXPECTED_WINDOW_FRAME(1099),
+ COMPILATION_UNEXPECTED_WINDOW_EXPRESSION(1100),
+ COMPILATION_UNEXPECTED_WINDOW_ORDERBY(1101),
+ COMPILATION_EXPECTED_WINDOW_FUNCTION(1102),
+ COMPILATION_ILLEGAL_USE_OF_IDENTIFIER(1103),
+ INVALID_FUNCTION_MODIFIER(1104),
+ OPERATION_NOT_SUPPORTED_ON_PRIMARY_INDEX(1105),
+ EXPECTED_CONSTANT_VALUE(1106),
+ UNEXPECTED_HINT(1107),
+ EXTERNAL_SOURCE_ERROR(1108),
+ EXTERNAL_SOURCE_CONTAINER_NOT_FOUND(1109),
+ PARAMETERS_NOT_ALLOWED_AT_SAME_TIME(1110),
+ PROPERTY_INVALID_VALUE_TYPE(1111),
+ INVALID_PROPERTY_FORMAT(1112),
+ INVALID_REGEX_PATTERN(1113),
+ EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES(1114),
+ INVALID_DATABASE_OBJECT_NAME(1115),
// Feed errors
- public static final int DATAFLOW_ILLEGAL_STATE = 3001;
- public static final int UTIL_DATAFLOW_UTILS_TUPLE_TOO_LARGE = 3002;
- public static final int UTIL_DATAFLOW_UTILS_UNKNOWN_FORWARD_POLICY = 3003;
- public static final int OPERATORS_FEED_INTAKE_OPERATOR_DESCRIPTOR_CLASSLOADER_NOT_CONFIGURED = 3004;
- public static final int PARSER_DELIMITED_NONOPTIONAL_NULL = 3005;
- public static final int PARSER_DELIMITED_ILLEGAL_FIELD = 3006;
- public static final int ADAPTER_TWITTER_TWITTER4J_LIB_NOT_FOUND = 3007;
- public static final int OPERATORS_FEED_INTAKE_OPERATOR_NODE_PUSHABLE_FAIL_AT_INGESTION = 3008;
- public static final int FEED_CREATE_FEED_DATATYPE_ERROR = 3009;
- public static final int PARSER_HIVE_NON_PRIMITIVE_LIST_NOT_SUPPORT = 3010;
- public static final int PARSER_HIVE_FIELD_TYPE = 3011;
- public static final int PARSER_HIVE_GET_COLUMNS = 3012;
- public static final int PARSER_HIVE_NO_CLOSED_COLUMNS = 3013;
- public static final int PARSER_HIVE_NOT_SUPPORT_NON_OP_UNION = 3014;
- public static final int PARSER_HIVE_MISSING_FIELD_TYPE_INFO = 3015;
- public static final int PARSER_HIVE_NULL_FIELD = 3016;
- public static final int PARSER_HIVE_NULL_VALUE_IN_LIST = 3017;
- public static final int INPUT_RECORD_RECORD_WITH_METADATA_AND_PK_NULL_IN_NON_OPTIONAL = 3018;
- public static final int INPUT_RECORD_RECORD_WITH_METADATA_AND_PK_CANNT_GET_PKEY = 3019;
- public static final int FEED_CHANGE_FEED_CONNECTIVITY_ON_ALIVE_FEED = 3020;
- public static final int RECORD_READER_MALFORMED_INPUT_STREAM = 3021;
- public static final int PROVIDER_DATAFLOW_CONTROLLER_UNKNOWN_DATA_SOURCE = 3022;
- public static final int PROVIDER_DATASOURCE_FACTORY_UNKNOWN_INPUT_STREAM_FACTORY = 3023;
- public static final int UTIL_EXTERNAL_DATA_UTILS_FAIL_CREATE_STREAM_FACTORY = 3024;
- public static final int UNKNOWN_RECORD_READER_FACTORY = 3025;
- public static final int PROVIDER_STREAM_RECORD_READER_UNKNOWN_FORMAT = 3026;
- public static final int UNKNOWN_RECORD_FORMAT_FOR_META_PARSER = 3027;
- public static final int LIBRARY_JAVA_JOBJECTS_FIELD_ALREADY_DEFINED = 3028;
- public static final int LIBRARY_JAVA_JOBJECTS_UNKNOWN_FIELD = 3029;
- public static final int NODE_RESOLVER_NO_NODE_CONTROLLERS = 3031;
- public static final int NODE_RESOLVER_UNABLE_RESOLVE_HOST = 3032;
- public static final int INPUT_RECORD_CONVERTER_DCP_MSG_TO_RECORD_CONVERTER_UNKNOWN_DCP_REQUEST = 3033;
- public static final int FEED_DATAFLOW_FRAME_DISTR_REGISTER_FAILED_DATA_PROVIDER = 3034;
- public static final int INPUT_RECORD_READER_CHAR_ARRAY_RECORD_TOO_LARGE = 3038;
- public static final int LIBRARY_JOBJECT_ACCESSOR_CANNOT_PARSE_TYPE = 3039;
- public static final int LIBRARY_JOBJECT_UTIL_ILLEGAL_ARGU_TYPE = 3040;
- public static final int LIBRARY_EXTERNAL_FUNCTION_UNABLE_TO_LOAD_CLASS = 3041;
- public static final int LIBRARY_EXTERNAL_FUNCTION_UNSUPPORTED_KIND = 3042;
- public static final int LIBRARY_EXTERNAL_FUNCTION_UNKNOWN_KIND = 3043;
- public static final int LIBRARY_EXTERNAL_LIBRARY_CLASS_REGISTERED = 3044;
- public static final int LIBRARY_JAVA_FUNCTION_HELPER_CANNOT_HANDLE_ARGU_TYPE = 3045;
- public static final int LIBRARY_JAVA_FUNCTION_HELPER_OBJ_TYPE_NOT_SUPPORTED = 3046;
- public static final int LIBRARY_EXTERNAL_FUNCTION_UNSUPPORTED_NAME = 3047;
- public static final int OPERATORS_FEED_META_OPERATOR_DESCRIPTOR_INVALID_RUNTIME = 3048;
- public static final int PARSER_FACTORY_DELIMITED_DATA_PARSER_FACTORY_NOT_VALID_DELIMITER = 3049;
- public static final int PARSER_FACTORY_DELIMITED_DATA_PARSER_FACTORY_NOT_VALID_QUOTE = 3050;
- public static final int PARSER_FACTORY_DELIMITED_DATA_PARSER_FACTORY_QUOTE_DELIMITER_MISMATCH = 3051;
- public static final int INDEXING_EXTERNAL_FILE_INDEX_ACCESSOR_UNABLE_TO_FIND_FILE_INDEX = 3052;
- public static final int PARSER_ADM_DATA_PARSER_FIELD_NOT_NULL = 3053;
- public static final int PARSER_ADM_DATA_PARSER_TYPE_MISMATCH = 3054;
- public static final int PARSER_ADM_DATA_PARSER_UNEXPECTED_TOKEN_KIND = 3055;
- public static final int PARSER_ADM_DATA_PARSER_ILLEGAL_ESCAPE = 3056;
- public static final int PARSER_ADM_DATA_PARSER_RECORD_END_UNEXPECTED = 3057;
- public static final int PARSER_ADM_DATA_PARSER_EXTRA_FIELD_IN_CLOSED_RECORD = 3058;
- public static final int PARSER_ADM_DATA_PARSER_UNEXPECTED_TOKEN_WHEN_EXPECT_COMMA = 3059;
- public static final int PARSER_ADM_DATA_PARSER_FOUND_COMMA_WHEN = 3060;
- public static final int PARSER_ADM_DATA_PARSER_UNSUPPORTED_INTERVAL_TYPE = 3061;
- public static final int PARSER_ADM_DATA_PARSER_INTERVAL_NOT_CLOSED = 3062;
- public static final int PARSER_ADM_DATA_PARSER_INTERVAL_BEGIN_END_POINT_MISMATCH = 3063;
- public static final int PARSER_ADM_DATA_PARSER_INTERVAL_MISSING_COMMA = 3064;
- public static final int PARSER_ADM_DATA_PARSER_INTERVAL_INVALID_DATETIME = 3065;
- public static final int PARSER_ADM_DATA_PARSER_INTERVAL_UNSUPPORTED_TYPE = 3066;
- public static final int PARSER_ADM_DATA_PARSER_INTERVAL_INTERVAL_ARGUMENT_ERROR = 3067;
- public static final int PARSER_ADM_DATA_PARSER_LIST_FOUND_END_COLLECTION = 3068;
- public static final int PARSER_ADM_DATA_PARSER_LIST_FOUND_COMMA_BEFORE_LIST = 3069;
- public static final int PARSER_ADM_DATA_PARSER_LIST_FOUND_COMMA_EXPECTING_ITEM = 3070;
- public static final int PARSER_ADM_DATA_PARSER_LIST_FOUND_END_RECOD = 3071;
- public static final int PARSER_ADM_DATA_PARSER_CAST_ERROR = 3072;
- public static final int PARSER_ADM_DATA_PARSER_CONSTRUCTOR_MISSING_DESERIALIZER = 3073;
- public static final int PARSER_ADM_DATA_PARSER_WRONG_INSTANCE = 3074;
- public static final int PARSER_TWEET_PARSER_CLOSED_FIELD_NULL = 3075;
- public static final int UTIL_FILE_SYSTEM_WATCHER_NO_FILES_FOUND = 3076;
- public static final int UTIL_LOCAL_FILE_SYSTEM_UTILS_PATH_NOT_FOUND = 3077;
- public static final int UTIL_HDFS_UTILS_CANNOT_OBTAIN_HDFS_SCHEDULER = 3078;
- public static final int ACTIVE_MANAGER_SHUTDOWN = 3079;
- public static final int FEED_METADATA_UTIL_UNEXPECTED_FEED_DATATYPE = 3080;
- public static final int FEED_METADATA_SOCKET_ADAPTOR_SOCKET_NOT_PROPERLY_CONFIGURED = 3081;
- public static final int FEED_METADATA_SOCKET_ADAPTOR_SOCKET_INVALID_HOST_NC = 3082;
- public static final int PROVIDER_DATASOURCE_FACTORY_DUPLICATE_FORMAT_MAPPING = 3083;
- public static final int CANNOT_SUBSCRIBE_TO_FAILED_ACTIVE_ENTITY = 3084;
- public static final int FEED_UNKNOWN_ADAPTER_NAME = 3085;
- public static final int PROVIDER_STREAM_RECORD_READER_WRONG_CONFIGURATION = 3086;
- public static final int FEED_CONNECT_FEED_APPLIED_INVALID_FUNCTION = 3087;
- public static final int ACTIVE_MANAGER_INVALID_RUNTIME = 3088;
- public static final int ACTIVE_ENTITY_ALREADY_STARTED = 3089;
- public static final int ACTIVE_ENTITY_CANNOT_BE_STOPPED = 3090;
- public static final int CANNOT_ADD_DATASET_TO_ACTIVE_ENTITY = 3091;
- public static final int CANNOT_REMOVE_DATASET_FROM_ACTIVE_ENTITY = 3092;
- public static final int ACTIVE_ENTITY_IS_ALREADY_REGISTERED = 3093;
- public static final int CANNOT_ADD_INDEX_TO_DATASET_CONNECTED_TO_ACTIVE_ENTITY = 3094;
- public static final int CANNOT_REMOVE_INDEX_FROM_DATASET_CONNECTED_TO_ACTIVE_ENTITY = 3095;
- public static final int ACTIVE_NOTIFICATION_HANDLER_IS_SUSPENDED = 3096;
- public static final int ACTIVE_ENTITY_LISTENER_IS_NOT_REGISTERED = 3097;
- public static final int CANNOT_DERIGESTER_ACTIVE_ENTITY_LISTENER = 3098;
- public static final int DOUBLE_INITIALIZATION_OF_ACTIVE_NOTIFICATION_HANDLER = 3099;
- public static final int DOUBLE_RECOVERY_ATTEMPTS = 3101;
- public static final int UNREPORTED_TASK_FAILURE_EXCEPTION = 3102;
- public static final int ACTIVE_ENTITY_ALREADY_SUSPENDED = 3103;
- public static final int ACTIVE_ENTITY_CANNOT_RESUME_FROM_STATE = 3104;
- public static final int ACTIVE_RUNTIME_IS_ALREADY_REGISTERED = 3105;
- public static final int ACTIVE_RUNTIME_IS_NOT_REGISTERED = 3106;
- public static final int ACTIVE_EVENT_HANDLER_ALREADY_SUSPENDED = 3107;
- public static final int METADATA_DROP_FUCTION_IN_USE = 3109;
- public static final int FEED_FAILED_WHILE_GETTING_A_NEW_RECORD = 3110;
- public static final int FEED_START_FEED_WITHOUT_CONNECTION = 3111;
- public static final int PARSER_COLLECTION_ITEM_CANNOT_BE_NULL = 3112;
- public static final int FAILED_TO_PARSE_RECORD = 3113;
- public static final int FAILED_TO_PARSE_RECORD_CONTENT = 3114;
- public static final int FAILED_TO_PARSE_METADATA = 3115;
- public static final int INPUT_DECODE_FAILURE = 3116;
- public static final int FAILED_TO_PARSE_MALFORMED_LOG_RECORD = 3117;
+ DATAFLOW_ILLEGAL_STATE(3001),
+ UTIL_DATAFLOW_UTILS_TUPLE_TOO_LARGE(3002),
+ UTIL_DATAFLOW_UTILS_UNKNOWN_FORWARD_POLICY(3003),
+ OPERATORS_FEED_INTAKE_OPERATOR_DESCRIPTOR_CLASSLOADER_NOT_CONFIGURED(3004),
+ PARSER_DELIMITED_NONOPTIONAL_NULL(3005),
+ PARSER_DELIMITED_ILLEGAL_FIELD(3006),
+ ADAPTER_TWITTER_TWITTER4J_LIB_NOT_FOUND(3007),
+ OPERATORS_FEED_INTAKE_OPERATOR_NODE_PUSHABLE_FAIL_AT_INGESTION(3008),
+ FEED_CREATE_FEED_DATATYPE_ERROR(3009),
+ PARSER_HIVE_NON_PRIMITIVE_LIST_NOT_SUPPORT(3010),
+ PARSER_HIVE_FIELD_TYPE(3011),
+ PARSER_HIVE_GET_COLUMNS(3012),
+ PARSER_HIVE_NO_CLOSED_COLUMNS(3013),
+ PARSER_HIVE_NOT_SUPPORT_NON_OP_UNION(3014),
+ PARSER_HIVE_MISSING_FIELD_TYPE_INFO(3015),
+ PARSER_HIVE_NULL_FIELD(3016),
+ PARSER_HIVE_NULL_VALUE_IN_LIST(3017),
+ INPUT_RECORD_RECORD_WITH_METADATA_AND_PK_NULL_IN_NON_OPTIONAL(3018),
+ INPUT_RECORD_RECORD_WITH_METADATA_AND_PK_CANNT_GET_PKEY(3019),
+ FEED_CHANGE_FEED_CONNECTIVITY_ON_ALIVE_FEED(3020),
+ RECORD_READER_MALFORMED_INPUT_STREAM(3021),
+ PROVIDER_DATAFLOW_CONTROLLER_UNKNOWN_DATA_SOURCE(3022),
+ PROVIDER_DATASOURCE_FACTORY_UNKNOWN_INPUT_STREAM_FACTORY(3023),
+ UTIL_EXTERNAL_DATA_UTILS_FAIL_CREATE_STREAM_FACTORY(3024),
+ UNKNOWN_RECORD_READER_FACTORY(3025),
+ PROVIDER_STREAM_RECORD_READER_UNKNOWN_FORMAT(3026),
+ UNKNOWN_RECORD_FORMAT_FOR_META_PARSER(3027),
+ LIBRARY_JAVA_JOBJECTS_FIELD_ALREADY_DEFINED(3028),
+ LIBRARY_JAVA_JOBJECTS_UNKNOWN_FIELD(3029),
+ NODE_RESOLVER_NO_NODE_CONTROLLERS(3031),
+ NODE_RESOLVER_UNABLE_RESOLVE_HOST(3032),
+ INPUT_RECORD_CONVERTER_DCP_MSG_TO_RECORD_CONVERTER_UNKNOWN_DCP_REQUEST(3033),
+ FEED_DATAFLOW_FRAME_DISTR_REGISTER_FAILED_DATA_PROVIDER(3034),
+ INPUT_RECORD_READER_CHAR_ARRAY_RECORD_TOO_LARGE(3038),
+ LIBRARY_JOBJECT_ACCESSOR_CANNOT_PARSE_TYPE(3039),
+ LIBRARY_JOBJECT_UTIL_ILLEGAL_ARGU_TYPE(3040),
+ LIBRARY_EXTERNAL_FUNCTION_UNABLE_TO_LOAD_CLASS(3041),
+ LIBRARY_EXTERNAL_FUNCTION_UNSUPPORTED_KIND(3042),
+ LIBRARY_EXTERNAL_FUNCTION_UNKNOWN_KIND(3043),
+ LIBRARY_EXTERNAL_LIBRARY_CLASS_REGISTERED(3044),
+ LIBRARY_JAVA_FUNCTION_HELPER_CANNOT_HANDLE_ARGU_TYPE(3045),
+ LIBRARY_JAVA_FUNCTION_HELPER_OBJ_TYPE_NOT_SUPPORTED(3046),
+ LIBRARY_EXTERNAL_FUNCTION_UNSUPPORTED_NAME(3047),
+ OPERATORS_FEED_META_OPERATOR_DESCRIPTOR_INVALID_RUNTIME(3048),
+ INVALID_DELIMITER(3049),
+ INVALID_CHAR_LENGTH(3050),
+ QUOTE_DELIMITER_MISMATCH(3051),
+ INDEXING_EXTERNAL_FILE_INDEX_ACCESSOR_UNABLE_TO_FIND_FILE_INDEX(3052),
+ PARSER_ADM_DATA_PARSER_FIELD_NOT_NULL(3053),
+ PARSER_ADM_DATA_PARSER_TYPE_MISMATCH(3054),
+ PARSER_ADM_DATA_PARSER_UNEXPECTED_TOKEN_KIND(3055),
+ PARSER_ADM_DATA_PARSER_ILLEGAL_ESCAPE(3056),
+ PARSER_ADM_DATA_PARSER_RECORD_END_UNEXPECTED(3057),
+ PARSER_ADM_DATA_PARSER_EXTRA_FIELD_IN_CLOSED_RECORD(3058),
+ PARSER_ADM_DATA_PARSER_UNEXPECTED_TOKEN_WHEN_EXPECT_COMMA(3059),
+ PARSER_ADM_DATA_PARSER_FOUND_COMMA_WHEN(3060),
+ PARSER_ADM_DATA_PARSER_UNSUPPORTED_INTERVAL_TYPE(3061),
+ PARSER_ADM_DATA_PARSER_INTERVAL_NOT_CLOSED(3062),
+ PARSER_ADM_DATA_PARSER_INTERVAL_BEGIN_END_POINT_MISMATCH(3063),
+ PARSER_ADM_DATA_PARSER_INTERVAL_MISSING_COMMA(3064),
+ PARSER_ADM_DATA_PARSER_INTERVAL_INVALID_DATETIME(3065),
+ PARSER_ADM_DATA_PARSER_INTERVAL_UNSUPPORTED_TYPE(3066),
+ PARSER_ADM_DATA_PARSER_INTERVAL_INTERVAL_ARGUMENT_ERROR(3067),
+ PARSER_ADM_DATA_PARSER_LIST_FOUND_END_COLLECTION(3068),
+ PARSER_ADM_DATA_PARSER_LIST_FOUND_COMMA_BEFORE_LIST(3069),
+ PARSER_ADM_DATA_PARSER_LIST_FOUND_COMMA_EXPECTING_ITEM(3070),
+ PARSER_ADM_DATA_PARSER_LIST_FOUND_END_RECOD(3071),
+ PARSER_ADM_DATA_PARSER_CAST_ERROR(3072),
+ PARSER_ADM_DATA_PARSER_CONSTRUCTOR_MISSING_DESERIALIZER(3073),
+ PARSER_ADM_DATA_PARSER_WRONG_INSTANCE(3074),
+ PARSER_TWEET_PARSER_CLOSED_FIELD_NULL(3075),
+ UTIL_FILE_SYSTEM_WATCHER_NO_FILES_FOUND(3076),
+ UTIL_LOCAL_FILE_SYSTEM_UTILS_PATH_NOT_FOUND(3077),
+ UTIL_HDFS_UTILS_CANNOT_OBTAIN_HDFS_SCHEDULER(3078),
+ ACTIVE_MANAGER_SHUTDOWN(3079),
+ FEED_METADATA_UTIL_UNEXPECTED_FEED_DATATYPE(3080),
+ FEED_METADATA_SOCKET_ADAPTOR_SOCKET_NOT_PROPERLY_CONFIGURED(3081),
+ FEED_METADATA_SOCKET_ADAPTOR_SOCKET_INVALID_HOST_NC(3082),
+ PROVIDER_DATASOURCE_FACTORY_DUPLICATE_FORMAT_MAPPING(3083),
+ CANNOT_SUBSCRIBE_TO_FAILED_ACTIVE_ENTITY(3084),
+ FEED_UNKNOWN_ADAPTER_NAME(3085),
+ PROVIDER_STREAM_RECORD_READER_WRONG_CONFIGURATION(3086),
+ FEED_CONNECT_FEED_APPLIED_INVALID_FUNCTION(3087),
+ ACTIVE_MANAGER_INVALID_RUNTIME(3088),
+ ACTIVE_ENTITY_ALREADY_STARTED(3089),
+ ACTIVE_ENTITY_CANNOT_BE_STOPPED(3090),
+ CANNOT_ADD_DATASET_TO_ACTIVE_ENTITY(3091),
+ CANNOT_REMOVE_DATASET_FROM_ACTIVE_ENTITY(3092),
+ ACTIVE_ENTITY_IS_ALREADY_REGISTERED(3093),
+ CANNOT_ADD_INDEX_TO_DATASET_CONNECTED_TO_ACTIVE_ENTITY(3094),
+ CANNOT_REMOVE_INDEX_FROM_DATASET_CONNECTED_TO_ACTIVE_ENTITY(3095),
+ ACTIVE_NOTIFICATION_HANDLER_IS_SUSPENDED(3096),
+ ACTIVE_ENTITY_LISTENER_IS_NOT_REGISTERED(3097),
+ CANNOT_DERIGESTER_ACTIVE_ENTITY_LISTENER(3098),
+ DOUBLE_INITIALIZATION_OF_ACTIVE_NOTIFICATION_HANDLER(3099),
+ DOUBLE_RECOVERY_ATTEMPTS(3101),
+ UNREPORTED_TASK_FAILURE_EXCEPTION(3102),
+ ACTIVE_ENTITY_ALREADY_SUSPENDED(3103),
+ ACTIVE_ENTITY_CANNOT_RESUME_FROM_STATE(3104),
+ ACTIVE_RUNTIME_IS_ALREADY_REGISTERED(3105),
+ ACTIVE_RUNTIME_IS_NOT_REGISTERED(3106),
+ ACTIVE_EVENT_HANDLER_ALREADY_SUSPENDED(3107),
+ METADATA_DROP_FUCTION_IN_USE(3109),
+ FEED_FAILED_WHILE_GETTING_A_NEW_RECORD(3110),
+ FEED_START_FEED_WITHOUT_CONNECTION(3111),
+ PARSER_COLLECTION_ITEM_CANNOT_BE_NULL(3112),
+ FAILED_TO_PARSE_RECORD(3113),
+ FAILED_TO_PARSE_RECORD_CONTENT(3114),
+ FAILED_TO_PARSE_METADATA(3115),
+ INPUT_DECODE_FAILURE(3116),
+ FAILED_TO_PARSE_MALFORMED_LOG_RECORD(3117),
+ ACTIVE_ENTITY_NOT_RUNNING(3118),
+ REQUIRED_PARAM_IF_PARAM_IS_PRESENT(3119),
// Lifecycle management errors
- public static final int DUPLICATE_PARTITION_ID = 4000;
+ DUPLICATE_PARTITION_ID(4000),
// Extension errors
- public static final int EXTENSION_ID_CONFLICT = 4001;
- public static final int EXTENSION_COMPONENT_CONFLICT = 4002;
- public static final int UNSUPPORTED_MESSAGE_TYPE = 4003;
- public static final int INVALID_CONFIGURATION = 4004;
- public static final int UNSUPPORTED_REPLICATION_STRATEGY = 4005;
+ EXTENSION_ID_CONFLICT(4001),
+ EXTENSION_COMPONENT_CONFLICT(4002),
+ UNSUPPORTED_MESSAGE_TYPE(4003),
+ INVALID_CONFIGURATION(4004),
+ UNSUPPORTED_REPLICATION_STRATEGY(4005),
// Lifecycle management errors pt.2
- public static final int CLUSTER_STATE_UNUSABLE = 4006;
+ CLUSTER_STATE_UNUSABLE(4006),;
- private ErrorCode() {
+ private static final String RESOURCE_PATH = "asx_errormsg/en.properties";
+ public static final String ASTERIX = "ASX";
+ private final int intValue;
+ private volatile String message;
+
+ ErrorCode(int intValue) {
+ this.intValue = intValue;
}
- private static class Holder {
- private static final Map<Integer, String> errorMessageMap;
-
- static {
- // Loads the map that maps error codes to error message templates.
- try (InputStream resourceStream = ErrorCode.class.getClassLoader().getResourceAsStream(RESOURCE_PATH)) {
- errorMessageMap = ErrorMessageUtil.loadErrorMap(resourceStream);
- } catch (Exception e) {
- throw new IllegalStateException(e);
- }
- }
-
- private Holder() {
- }
+ @Override
+ public String component() {
+ return ASTERIX;
}
- public static String getErrorMessage(int errorCode) {
- String msg = Holder.errorMessageMap.get(errorCode);
- if (msg == null) {
- throw new IllegalStateException("Undefined error code: " + errorCode);
+ @Override
+ public int intValue() {
+ return intValue;
+ }
+
+ @Override
+ public String errorMessage() {
+ return ErrorMessageMapHolder.get(this);
+ }
+
+ private static class ErrorMessageMapHolder {
+ private static final String[] enumMessages =
+ ErrorMessageUtil.defineMessageEnumOrdinalMap(values(), RESOURCE_PATH);
+
+ private static String get(ErrorCode errorCode) {
+ return enumMessages[errorCode.ordinal()];
}
- return msg;
}
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ExceptionUtils.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ExceptionUtils.java
index 359054e..f73848f 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ExceptionUtils.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ExceptionUtils.java
@@ -20,7 +20,7 @@
import java.util.function.Predicate;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IFormattedException;
public class ExceptionUtils {
public static final String INCORRECT_PARAMETER = "Incorrect parameter.\n";
@@ -53,22 +53,6 @@
return current;
}
- public static Throwable getCause(Throwable e, String component, int code) {
- Throwable current = e;
- Throwable expected =
- (current instanceof HyracksDataException && ((HyracksDataException) current).getErrorCode() == code
- && ((HyracksDataException) current).getComponent().equals(component)) ? current : null;
- Throwable cause = e.getCause();
- while (cause != null && cause != current) {
- current = cause;
- expected =
- (current instanceof HyracksDataException && ((HyracksDataException) current).getErrorCode() == code
- && ((HyracksDataException) current).getComponent().equals(component)) ? current : expected;
- cause = current.getCause();
- }
- return expected == null ? current : expected;
- }
-
/**
* Determines whether supplied exception contains a matching cause in its hierarchy, or is itself a match
*/
@@ -85,4 +69,18 @@
}
return test.test(e);
}
+
+ /**
+ * Unwraps enclosed exceptions until a non-product exception is found, otherwise returns the root production
+ * exception
+ */
+ public static Throwable unwrap(Throwable e) {
+ Throwable current = e;
+ Throwable cause = e.getCause();
+ while (cause != null && cause != current && current instanceof IFormattedException) {
+ current = cause;
+ cause = current.getCause();
+ }
+ return current;
+ }
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/MetadataException.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/MetadataException.java
index a735c81..13a006d 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/MetadataException.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/MetadataException.java
@@ -30,51 +30,51 @@
private static final long serialVersionUID = 1L;
private static final Logger LOGGER = LogManager.getLogger();
- @Deprecated
/**
- * @Deprecated Instead, use a constructor with error code
+ * @deprecated Instead, use a constructor with error code
* @param message
*/
+ @Deprecated
public MetadataException(String message) {
super(message);
}
- @Deprecated
/**
- * @Deprecated When creating a constructor with cause,
+ * @deprecated When creating a constructor with cause,
* create AlgebricksException using AlgebricksException.create(Throwable th);
* @param cause
*/
+ @Deprecated
public MetadataException(Throwable cause) {
super(cause);
}
- @Deprecated
/**
- * @Deprecated When creating a constructor with cause,
+ * @deprecated When creating a constructor with cause,
* create AlgebricksException using AlgebricksException.create(Throwable th);
* @param cause
*/
+ @Deprecated
public MetadataException(String message, Throwable cause) {
super(message, cause);
}
- public MetadataException(int errorCode, Serializable... params) {
- super(errorCode, params);
- }
-
- public MetadataException(int errorCode, SourceLocation sourceLoc, Serializable... params) {
- super(errorCode, sourceLoc, params);
- }
-
- public MetadataException(int errorCode, Throwable cause, Serializable... params) {
- super(errorCode, cause, params);
- }
-
- public MetadataException(int errorCode, Throwable cause, SourceLocation sourceLoc, Serializable... params) {
+ public MetadataException(ErrorCode errorCode, Throwable cause, SourceLocation sourceLoc, Serializable... params) {
super(errorCode, cause, sourceLoc, params);
}
+ public MetadataException(ErrorCode errorCode, SourceLocation sourceLoc, Serializable... params) {
+ this(errorCode, null, sourceLoc, params);
+ }
+
+ public MetadataException(ErrorCode errorCode, Throwable cause, Serializable... params) {
+ this(errorCode, cause, null, params);
+ }
+
+ public MetadataException(ErrorCode errorCode, Serializable... params) {
+ this(errorCode, null, null, params);
+ }
+
public static MetadataException create(Throwable cause) {
if (cause instanceof MetadataException || cause == null) {
return (MetadataException) cause;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/NoOpWarningCollector.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/NoOpWarningCollector.java
new file mode 100644
index 0000000..a036b7e
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/NoOpWarningCollector.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.common.exceptions;
+
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.Warning;
+
+public final class NoOpWarningCollector implements IWarningCollector {
+
+ public static final IWarningCollector INSTANCE = new NoOpWarningCollector();
+
+ private NoOpWarningCollector() {
+ }
+
+ @Override
+ public void warn(Warning warning) {
+ // no-op
+ }
+
+ @Override
+ public boolean shouldWarn() {
+ return false;
+ }
+
+ @Override
+ public long getTotalWarningsCount() {
+ return 0;
+ }
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/RuntimeDataException.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/RuntimeDataException.java
index 1c6763f..1a0a61f 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/RuntimeDataException.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/RuntimeDataException.java
@@ -27,19 +27,25 @@
public class RuntimeDataException extends HyracksDataException {
private static final long serialVersionUID = 1L;
- public RuntimeDataException(int errorCode, Serializable... params) {
- super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), params);
+ public static RuntimeDataException create(ErrorCode error, Serializable... params) {
+ return new RuntimeDataException(error, params);
}
- public RuntimeDataException(int errorCode, SourceLocation sourceLoc, Serializable... params) {
- super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), null, sourceLoc, params);
+ public RuntimeDataException(ErrorCode errorCode, Throwable cause, SourceLocation sourceLoc,
+ Serializable... params) {
+ super(errorCode, cause, sourceLoc, params);
}
- public RuntimeDataException(int errorCode, Throwable cause, Serializable... params) {
- super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), cause, params);
+ public RuntimeDataException(ErrorCode errorCode, Serializable... params) {
+ this(errorCode, null, null, params);
}
- public RuntimeDataException(int errorCode, Throwable cause, SourceLocation sourceLoc, Serializable... params) {
- super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), cause, sourceLoc, params);
+ public RuntimeDataException(ErrorCode errorCode, SourceLocation sourceLoc, Serializable... params) {
+ this(errorCode, null, sourceLoc, params);
}
+
+ public RuntimeDataException(ErrorCode errorCode, Throwable cause, Serializable... params) {
+ this(errorCode, cause, null, params);
+ }
+
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/WarningUtil.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/WarningUtil.java
index 6a369ef..4bbb3a7 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/WarningUtil.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/WarningUtil.java
@@ -21,19 +21,22 @@
import java.io.Serializable;
import java.util.Collection;
+import org.apache.hyracks.api.exceptions.IError;
import org.apache.hyracks.api.exceptions.IWarningCollector;
import org.apache.hyracks.api.exceptions.SourceLocation;
import org.apache.hyracks.api.exceptions.Warning;
-import org.apache.hyracks.api.util.ErrorMessageUtil;
public class WarningUtil {
private WarningUtil() {
}
- public static Warning forAsterix(SourceLocation srcLocation, int code, Serializable... params) {
- return Warning.of(ErrorCode.ASTERIX, srcLocation, code, ErrorMessageUtil.formatMessage(ErrorCode.ASTERIX, code,
- ErrorCode.getErrorMessage(code), srcLocation, params));
+ /**
+ * @deprecated call {@link Warning#of(SourceLocation, IError, Serializable...)} directly
+ */
+ @Deprecated
+ public static Warning forAsterix(SourceLocation srcLocation, ErrorCode code, Serializable... params) {
+ return Warning.of(srcLocation, code, params);
}
/** Merges the warnings from the collection argument into the warning collector argument. */
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/external/IAdapterFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/external/IAdapterFactory.java
new file mode 100644
index 0000000..6d2df37
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/external/IAdapterFactory.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.external;
+
+import java.io.Serializable;
+import java.util.Map;
+
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.application.IServiceContext;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+
+/**
+ * Base interface for IGenericDatasetAdapterFactory and ITypedDatasetAdapterFactory.
+ * Acts as a marker interface indicating that the implementation provides functionality
+ * for creating an adapter.
+ */
+public interface IAdapterFactory extends Serializable {
+
+ /**
+ * Returns the display name corresponding to the Adapter type that is created by the factory.
+ *
+ * @return the display name
+ */
+ String getAlias();
+
+ /**
+ * Gets a list of partition constraints. A partition constraint can be a
+ * requirement to execute at a particular location or could be cardinality
+ * constraints indicating the number of instances that need to run in
+ * parallel. example, a IDatasourceAdapter implementation written for data
+ * residing on the local file system of a node cannot run on any other node
+ * and thus has a location partition constraint. The location partition
+ * constraint can be expressed as a node IP address or a node controller id.
+ * In the former case, the IP address is translated to a node controller id
+ * running on the node with the given IP address.
+ *
+ * @throws AlgebricksException
+ */
+ AlgebricksAbsolutePartitionConstraint getPartitionConstraint() throws AlgebricksException;
+
+ /**
+ * Creates an instance of IDatasourceAdapter.
+ *
+ * @param ctx HyracksTaskContext
+ * @param partition partition number
+ * @return An instance of IDatasourceAdapter.
+ * @throws Exception
+ */
+ IDataSourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws HyracksDataException;
+
+ /**
+ * Configure the adapter
+ *
+ * @param serviceContext
+ * @param configuration
+ * @param warningCollector warning collector
+ * @throws AlgebricksException
+ * @throws HyracksDataException
+ */
+ void configure(IServiceContext serviceContext, Map<String, String> configuration,
+ IWarningCollector warningCollector) throws HyracksDataException, AlgebricksException;
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/external/IAdapterFactoryService.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/external/IAdapterFactoryService.java
new file mode 100644
index 0000000..55e25b7
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/external/IAdapterFactoryService.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.external;
+
+@FunctionalInterface
+public interface IAdapterFactoryService {
+
+ /**
+ * Creates and returns and adapter factory
+ *
+ * @return adapter factory
+ */
+ IAdapterFactory createAdapterFactory();
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/external/IDataSourceAdapter.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/external/IDataSourceAdapter.java
new file mode 100644
index 0000000..8fc70b8
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/external/IDataSourceAdapter.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.common.external;
+
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * A super interface implemented by a data source adapter. An adapter can be a
+ * pull based or push based. This interface provides all common APIs that need
+ * to be implemented by each adapter irrespective of the the kind of
+ * adapter(pull or push).
+ */
+@FunctionalInterface
+public interface IDataSourceAdapter {
+
+ public enum AdapterType {
+ INTERNAL,
+ EXTERNAL
+ }
+
+ /**
+ * Triggers the adapter to begin ingesting data from the external source.
+ *
+ * @param partition
+ * The adapter could be running with a degree of parallelism.
+ * partition corresponds to the i'th parallel instance.
+ * @param writer
+ * The instance of frame writer that is used by the adapter to
+ * write frame to. Adapter packs the fetched bytes (from external source),
+ * packs them into frames and forwards the frames to an upstream receiving
+ * operator using the instance of IFrameWriter.
+ * @throws Exception
+ */
+ public void start(int partition, IFrameWriter writer) throws HyracksDataException, InterruptedException;
+
+ /**
+ * @return The number of processed tuples by this adapter
+ */
+ default long getProcessedTuples() {
+ throw new UnsupportedOperationException();
+ }
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/DatasetResourceReference.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/DatasetResourceReference.java
index dc37fc2..6fd1c6a 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/DatasetResourceReference.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/DatasetResourceReference.java
@@ -19,6 +19,7 @@
package org.apache.asterix.common.storage;
import java.nio.file.Paths;
+import java.util.Objects;
import org.apache.asterix.common.dataflow.DatasetLocalResource;
import org.apache.asterix.common.utils.StorageConstants;
@@ -35,6 +36,7 @@
}
public static DatasetResourceReference of(LocalResource localResource) {
+ Objects.requireNonNull(localResource);
return parse(localResource);
}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IndexCheckpoint.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IndexCheckpoint.java
index 3d0b9cb..878c94e 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IndexCheckpoint.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/storage/IndexCheckpoint.java
@@ -38,7 +38,7 @@
private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
private static final long INITIAL_CHECKPOINT_ID = 0;
// TODO(mblow): remove this marker & related logic once we no longer are able to read indexes prior to the fix
- private static final long HAS_NULL_MISSING_VALUES_FIX = -1;
+ private static final long HAS_NULL_MISSING_VALUES_FIX = -2;
private long id;
private long validComponentSequence;
private long lowWatermark;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
index bfe7963..8a5f34e 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
@@ -85,17 +85,6 @@
long getLocalMinFirstLSN() throws HyracksDataException;
/**
- * Replay the logs that belong to the passed {@code partitions} starting from the {@code lowWaterMarkLSN}
- *
- * @param partitions
- * @param lowWaterMarkLSN
- * @throws IOException
- * @throws ACIDException
- */
- void replayPartitionsLogs(Set<Integer> partitions, ILogReader logReader, long lowWaterMarkLSN)
- throws IOException, ACIDException;
-
- /**
* Creates a temporary file to be used during recovery
*
* @param txnId
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index 457206d..e3acf9c 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -78,11 +78,13 @@
41 = Request %1$s has been cancelled
42 = %1$s: \"%2$s\" is not a TPC-DS table name
43 = Value out of range, function %1$s expects its %2$s input parameter value to be between %3$s and %4$s, received %5$s
-44 = %1$s statement is prohibited by this request
+44 = %1$s statement is not supported in read-only mode
45 = Invalid value: function %1$s expects its %2$s input parameter to be an integer value, got %3$s
46 = Invalid pattern \"%1$s\" for LIKE
47 = Invalid value for parameter \"%1$s\": %2$s
48 = Unable to process JSON content in request
+49 = Parameter(s) %1$s must be specified
+50 = Invalid parameter \"%1$s\"
100 = Unsupported JRE: %1$s
@@ -189,6 +191,14 @@
1105 = Operation not supported on primary index %1$s
1106 = Expected constant value
1107 = Unexpected hint: %1$s. %2$s expected at this location
+1108 = External source error. %1$s
+1109 = External source container %1$s not found
+1110 = The parameters \"%1$s\" and \"%2$s\" cannot be provided at the same time
+1111 = Property \"%1$s\" expects value(s) of type %2$s
+1112 = Invalid format for property \"%1$s\"
+1113 = Invalid pattern %1$s
+1114 = The provided external dataset configuration returned no files from the external source
+1115 = Invalid name for a database object: \"%1$s\"
# Feed Errors
3001 = Illegal state.
@@ -227,7 +237,7 @@
3035 = Feed already has an intake job
3036 = Feed job already registered in intake jobs
3037 = Feed job already registered in all jobs
-3038 = Record is too large!. Maximum record size is %1$s
+3038 = Record is too large. Maximum record size is %1$s
3039 = Cannot parse list item of type %1$s
3040 = Argument type: %1$s
3041 = Unable to load/instantiate class %1$s
@@ -239,7 +249,7 @@
3047 = External %1$s not supported
3048 = Invalid feed runtime: %1$s
3049 = '%1$s' is not a valid delimiter. The length of a delimiter should be 1
-3050 = '%1$s' is not a valid quote. The length of a quote should be 1
+3050 = '%1$s' is not a valid %2$s. The length of %2$s should be 1
3051 = Quote '%1$s' cannot be used with the delimiter '%2$s'
3052 = Was not able to find a file in the files index
3053 = Field %1$s can not be null
@@ -305,6 +315,8 @@
3115 = Failed to parse record metadata
3116 = Failed to decode input
3117 = Failed to parse record, malformed log record
+3118 = Active Entity %1$s is not running (it is %2$s)
+3119 = Parameter '%1$s' is required if '%2$s' is provided
# Lifecycle management errors
4000 = Partition id %1$s for node %2$s already in use by node %3$s
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/context/CorrelatedPrefixMergePolicyTest.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/context/CorrelatedPrefixMergePolicyTest.java
index 843cb21..779bfd8 100644
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/context/CorrelatedPrefixMergePolicyTest.java
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/context/CorrelatedPrefixMergePolicyTest.java
@@ -219,7 +219,7 @@
@Override
public Void answer(InvocationOnMock invocation) throws Throwable {
- List<ILSMDiskComponent> mergedComponents = invocation.getArgumentAt(0, List.class);
+ List<ILSMDiskComponent> mergedComponents = invocation.getArgument(0);
mergedComponents.forEach(component -> {
try {
resultComponentIDs.add(component.getId());
diff --git a/asterixdb/asterix-dashboard/pom.xml b/asterixdb/asterix-dashboard/pom.xml
index aae2902..e49bd3b 100644
--- a/asterixdb/asterix-dashboard/pom.xml
+++ b/asterixdb/asterix-dashboard/pom.xml
@@ -55,120 +55,131 @@
</dependency>
</dependencies>
- <build>
- <plugins>
- <plugin>
- <groupId>com.github.eirslett</groupId>
- <artifactId>frontend-maven-plugin</artifactId>
- <version>1.6</version>
- <configuration>
- <nodeVersion>v10.3.0</nodeVersion>
- <npmVersion>6.1.0</npmVersion>
- <workingDirectory>target/dashboard</workingDirectory>
- <installDirectory>target/dashboard</installDirectory>
- </configuration>
- <executions>
- <execution>
- <id>install node and yarn</id>
- <goals>
- <goal>install-node-and-npm</goal>
- </goals>
- <phase>generate-resources</phase>
- </execution>
- <execution>
- <id>npm install</id>
- <phase>process-resources</phase>
- <goals>
- <goal>npm</goal>
- </goals>
+ <profiles>
+ <profile>
+ <id>include-dashboard</id>
+ <activation>
+ <activeByDefault>true</activeByDefault>
+ </activation>
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>com.github.eirslett</groupId>
+ <artifactId>frontend-maven-plugin</artifactId>
+ <version>1.11.0</version>
<configuration>
- <arguments>install --cache ${settings.localRepository}/.npm-cache --no-optional</arguments>
+ <nodeVersion>v10.3.0</nodeVersion>
+ <npmVersion>6.1.0</npmVersion>
+ <workingDirectory>target/dashboard</workingDirectory>
+ <installDirectory>target/dashboard</installDirectory>
</configuration>
- </execution>
- <execution>
- <id>npm run-script mavenbuild</id>
- <phase>compile</phase>
- <goals>
- <goal>npm</goal>
- </goals>
+ <executions>
+ <execution>
+ <id>install node and yarn</id>
+ <goals>
+ <goal>install-node-and-npm</goal>
+ </goals>
+ <phase>generate-resources</phase>
+ </execution>
+ <execution>
+ <id>npm install</id>
+ <phase>process-resources</phase>
+ <goals>
+ <goal>npm</goal>
+ </goals>
+ <configuration>
+ <arguments>install --cache ${settings.localRepository}/.npm-cache --no-optional</arguments>
+ </configuration>
+ </execution>
+ <execution>
+ <id>npm run-script mavenbuild</id>
+ <phase>compile</phase>
+ <goals>
+ <goal>npm</goal>
+ </goals>
+ <configuration>
+ <arguments>run-script mavenbuild</arguments>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <artifactId>maven-resources-plugin</artifactId>
+ <executions>
+ <execution>
+ <id>copy-static</id>
+ <phase>${resource.stage}</phase>
+ <goals>
+ <goal>copy-resources</goal>
+ </goals>
+ <configuration>
+ <outputDirectory>
+ ${basedir}/target/classes/dashboard/
+ </outputDirectory>
+ <resources>
+ <resource>
+ <directory>${basedir}/target/dashboard/static/
+ </directory>
+ </resource>
+ </resources>
+ </configuration>
+ </execution>
+ <execution>
+ <id>copy-node</id>
+ <phase>validate</phase>
+ <goals>
+ <goal>copy-resources</goal>
+ </goals>
+ <configuration>
+ <outputDirectory>
+ ${basedir}/target/dashboard
+ </outputDirectory>
+ <resources>
+ <resource>
+ <directory>${basedir}/src/node
+ </directory>
+ </resource>
+ </resources>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>license-automation-plugin</artifactId>
+ <executions>
+ <execution>
+ <phase>${license.stage}</phase>
+ <goals>
+ <goal>generate</goal>
+ </goals>
+ </execution>
+ </executions>
<configuration>
- <arguments>run-script mavenbuild</arguments>
+ <templateRootDir>${basedir}</templateRootDir>
+ <outputDir>${project.build.directory}/classes/META-INF</outputDir>
+ <generatedFiles>
+ <generatedFile>
+ <template>src/main/licenses/dashboard-license.ftl</template>
+ <outputFile>LICENSE</outputFile>
+ </generatedFile>
+ </generatedFiles>
+ <location>repo/</location>
+ <timeoutSecs>10</timeoutSecs>
+ <downloadDir>
+ ${project.build.directory}/generated-resources/license
+ </downloadDir>
+ <excludedScopes>
+ <excludedScope>test</excludedScope>
+ </excludedScopes>
+ <licenseDirectory>${basedir}/src/main/licenses/content</licenseDirectory>
</configuration>
- </execution>
- </executions>
- </plugin>
- <plugin>
- <artifactId>maven-resources-plugin</artifactId>
- <executions>
- <execution>
- <id>copy-static</id>
- <phase>${resource.stage}</phase>
- <goals>
- <goal>copy-resources</goal>
- </goals>
- <configuration>
- <outputDirectory>
- ${basedir}/target/classes/dashboard/
- </outputDirectory>
- <resources>
- <resource>
- <directory>${basedir}/target/dashboard/static/
- </directory>
- </resource>
- </resources>
- </configuration>
- </execution>
- <execution>
- <id>copy-node</id>
- <phase>validate</phase>
- <goals>
- <goal>copy-resources</goal>
- </goals>
- <configuration>
- <outputDirectory>
- ${basedir}/target/dashboard
- </outputDirectory>
- <resources>
- <resource>
- <directory>${basedir}/src/node
- </directory>
- </resource>
- </resources>
- </configuration>
- </execution>
- </executions>
- </plugin>
- <plugin>
- <groupId>org.apache.hyracks</groupId>
- <artifactId>license-automation-plugin</artifactId>
- <executions>
- <execution>
- <phase>${license.stage}</phase>
- <goals>
- <goal>generate</goal>
- </goals>
- </execution>
- </executions>
- <configuration>
- <templateRootDir>${basedir}</templateRootDir>
- <outputDir>${project.build.directory}/classes/META-INF</outputDir>
- <generatedFiles>
- <generatedFile>
- <template>src/main/licenses/dashboard-license.ftl</template>
- <outputFile>LICENSE</outputFile>
- </generatedFile>
- </generatedFiles>
- <location>repo/</location>
- <timeoutSecs>10</timeoutSecs>
- <downloadDir>
- ${project.build.directory}/generated-resources/license
- </downloadDir>
- <excludedScopes>
- <excludedScope>test</excludedScope>
- </excludedScopes>
- <licenseDirectory>${basedir}/src/main/licenses/content</licenseDirectory>
- </configuration>
- </plugin>
- </plugins>
- </build>
+ </plugin>
+ </plugins>
+ </build>
+ </profile>
+ <profile>
+ <id>skip-dashboard</id>
+ </profile>
+ </profiles>
</project>
diff --git a/asterixdb/asterix-doc/pom.xml b/asterixdb/asterix-doc/pom.xml
index ea23f5e..e1a1e69 100644
--- a/asterixdb/asterix-doc/pom.xml
+++ b/asterixdb/asterix-doc/pom.xml
@@ -52,7 +52,7 @@
<configuration>
<target>
<concat destfile="${project.build.directory}/generated-site/markdown/sqlpp/manual.md">
- <filelist dir="${project.basedir}/src/main/markdown/sqlpp" files="0_toc.md,1_intro.md,2_expr_title.md,2_expr.md,3_query_title.md,3_declare_dataverse.md,3_declare_function.md,3_query.md,4_error_title.md,4_error.md,5_ddl_head.md,5_ddl_dataset_index.md,5_ddl_function_removal.md,5_ddl_dml.md,appendix_1_title.md,appendix_1_keywords.md,appendix_2_title.md,appendix_2_parameters.md,appendix_2_parallel_sort.md,appendix_2_index_only.md,appendix_3_title.md,appendix_3_resolution.md" />
+ <filelist dir="${project.basedir}/src/main/markdown/sqlpp" files="0_toc.md,1_intro.md,2_expr_title.md,2_expr.md,3_query_title.md,3_declare_dataverse.md,3_declare_function.md,3_query.md,4_error_title.md,4_error.md,5_ddl_head.md,5_ddl_dataset_index.md,5_ddl_function_removal.md,5_ddl_dml.md,appendix_1_title.md,appendix_1_keywords.md,appendix_2_title.md,appendix_2_parameters.md,appendix_2_parallel_sort.md,appendix_2_index_only.md,appendix_2_hints.md,appendix_3_title.md,appendix_3_resolution.md" />
</concat>
<concat destfile="${project.build.directory}/generated-site/markdown/sqlpp/builtins.md">
<filelist dir="${project.basedir}/src/main/markdown/builtins" files="0_toc.md,0_toc_sqlpp.md,0_toc_common.md,1_numeric_common.md,1_numeric_delta.md,2_string_common.md,2_string_delta.md,3_binary.md,4_spatial.md,5_similarity.md,6_tokenizing.md,7_temporal.md,7_allens.md,8_record.md,9_aggregate_sql.md,10_comparison.md,11_type.md,13_conditional.md,12_misc.md,15_bitwise.md,14_window.md" />
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/0_toc.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/0_toc.md
index 1a5a3e6..8680c43 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/0_toc.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/0_toc.md
@@ -99,4 +99,5 @@
* [Appendix 2. Performance Tuning](#Performance_tuning)
* [Parallelism Parameter](#Parallelism_parameter)
* [Memory Parameters](#Memory_parameters)
+ * [Query Hints](#Query_hints)
* [Appendix 3. Variable Bindings and Name Resolution](#Variable_bindings_and_name_resolution)
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/2_expr.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/2_expr.md
index 37e7f79..2e0b526 100644
--- a/asterixdb/asterix-doc/src/main/markdown/sqlpp/2_expr.md
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/2_expr.md
@@ -218,8 +218,9 @@
single element from an array, or a whole subset of an array. Accessing a single element is achieved by
providing a single index argument (zero-based element position), while obtaining a subset of an array is achieved by
providing the `start` and `end` (zero-based) index positions; the returned subset is from position `start` to position
-`end - 1`; the `end` position argument is optional. Multisets have similar behavior to arrays, except for retrieving
-arbitrary items as the order of items is not fixed in multisets.
+`end - 1`; the `end` position argument is optional. If a position argument is negative then the element position is
+counted from the end of the array (`-1` addresses the last element, `-2` next to last, and so on). Multisets have
+similar behavior to arrays, except for retrieving arbitrary items as the order of items is not fixed in multisets.
Attempts to access non-existent fields or out-of-bound array elements produce the special value `MISSING`. Type errors
will be raised for inappropriate use of a path expression, such as applying a field accessor to a numeric value.
@@ -232,12 +233,16 @@
({"name": "MyABCs", "array": [ "a", "b", "c"]}).array
(["a", "b", "c"])[2]
+
+ (["a", "b", "c"])[-1]
({"name": "MyABCs", "array": [ "a", "b", "c"]}).array[2]
(["a", "b", "c"])[0:2]
(["a", "b", "c"])[0:]
+
+ (["a", "b", "c"])[-2:-1]
## <a id="Primary_expressions">Primary Expressions</a>
diff --git a/asterixdb/asterix-doc/src/main/markdown/sqlpp/appendix_2_hints.md b/asterixdb/asterix-doc/src/main/markdown/sqlpp/appendix_2_hints.md
new file mode 100644
index 0000000..0e4f470
--- /dev/null
+++ b/asterixdb/asterix-doc/src/main/markdown/sqlpp/appendix_2_hints.md
@@ -0,0 +1,52 @@
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements. See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership. The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License. You may obtain a copy of the License at
+ !
+ ! http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied. See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+
+## <a id="Query_hints">Query Hints</a>
+
+#### <a id="hash_groupby">"hash" GROUP BY hint</a>
+
+The system supports two algorithms for GROUP BY clause evaluation: pre-sorted and hash-based.
+By default it uses the pre-sorted approach: The input data is first sorted on the grouping fields
+and then aggregation is performed on that sorted data. The alternative is a hash-based strategy
+which can be enabled via a `/*+ hash */` GROUP BY hint: The data is aggregated using an in-memory hash-table
+(that can spill to disk if necessary). This approach is recommended for low-cardinality grouping fields.
+
+##### Example:
+
+ SELECT c.address.state, count(*)
+ FROM Customers AS c
+ /*+ hash */ GROUP BY c.address.state
+
+#### <a id="hash_bcast_join">"hash-bcast" JOIN hint</a>
+
+By default the system uses a partitioned-parallel hash join strategy to parallelize the execution of an
+equi-join. In this approach both sides of the join are repartitioned (if necessary) on a hash of the join key;
+potentially matching data items thus arrive at the same partition to be joined locally.
+This strategy is robust, but not always the fastest when one of the join sides is low cardinality and
+the other is high cardinality (since it scans and potentially moves the data from both sides).
+This special case can be better handled by broadcasting (replicating) the smaller side to all data partitions
+of the larger side and not moving the data from the other (larger) side. The system provides a join hint to enable
+this strategy: `/*+ hash-bcast */`. This hint forces the right side of the join to be replicated while the left side
+retains its original partitioning.
+
+##### Example:
+
+ SELECT *
+ FROM Orders AS o JOIN Customers AS c
+ ON o.customer_id /*+ hash-bcast */ = c.customer_id
diff --git a/asterixdb/asterix-external-data/pom.xml b/asterixdb/asterix-external-data/pom.xml
index 1092422..611e58b 100644
--- a/asterixdb/asterix-external-data/pom.xml
+++ b/asterixdb/asterix-external-data/pom.xml
@@ -286,7 +286,7 @@
</dependency>
<dependency>
<groupId>org.mockito</groupId>
- <artifactId>mockito-all</artifactId>
+ <artifactId>mockito-core</artifactId>
<scope>test</scope>
</dependency>
<dependency>
@@ -439,5 +439,29 @@
<groupId>io.netty</groupId>
<artifactId>netty-codec-http</artifactId>
</dependency>
+ <dependency>
+ <groupId>software.amazon.awssdk</groupId>
+ <artifactId>aws-core</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>software.amazon.awssdk</groupId>
+ <artifactId>http-client-spi</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>software.amazon.awssdk</groupId>
+ <artifactId>sdk-core</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>software.amazon.awssdk</groupId>
+ <artifactId>s3</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>software.amazon.awssdk</groupId>
+ <artifactId>regions</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>software.amazon.awssdk</groupId>
+ <artifactId>auth</artifactId>
+ </dependency>
</dependencies>
</project>
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/AdapterFactoryService.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/AdapterFactoryService.java
new file mode 100644
index 0000000..aaf2002
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/AdapterFactoryService.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.adapter.factory;
+
+import org.apache.asterix.common.external.IAdapterFactoryService;
+import org.apache.asterix.external.api.ITypedAdapterFactory;
+
+public class AdapterFactoryService implements IAdapterFactoryService {
+
+ /**
+ * Creates and returns an adapter factory
+ *
+ * @return adaptor factory
+ */
+ @Override
+ public ITypedAdapterFactory createAdapterFactory() {
+ return new GenericAdapterFactory();
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/GenericAdapterFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/GenericAdapterFactory.java
index fc59f68..badf105 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/GenericAdapterFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/GenericAdapterFactory.java
@@ -25,14 +25,15 @@
import org.apache.asterix.common.api.IApplicationContext;
import org.apache.asterix.common.api.INcApplicationContext;
import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.external.IDataSourceAdapter;
import org.apache.asterix.common.library.ILibraryManager;
-import org.apache.asterix.external.api.IAdapterFactory;
import org.apache.asterix.external.api.IDataFlowController;
import org.apache.asterix.external.api.IDataParserFactory;
-import org.apache.asterix.external.api.IDataSourceAdapter;
import org.apache.asterix.external.api.IExternalDataSourceFactory;
import org.apache.asterix.external.api.IIndexibleExternalDataSource;
import org.apache.asterix.external.api.IIndexingAdapterFactory;
+import org.apache.asterix.external.api.ITypedAdapterFactory;
import org.apache.asterix.external.dataflow.AbstractFeedDataFlowController;
import org.apache.asterix.external.dataset.adapter.FeedAdapter;
import org.apache.asterix.external.dataset.adapter.GenericAdapter;
@@ -54,12 +55,13 @@
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
import org.apache.hyracks.api.io.FileSplit;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
-public class GenericAdapterFactory implements IIndexingAdapterFactory, IAdapterFactory {
+public class GenericAdapterFactory implements IIndexingAdapterFactory, ITypedAdapterFactory {
private static final long serialVersionUID = 1L;
private static final Logger LOGGER = LogManager.getLogger();
@@ -99,7 +101,7 @@
INCServiceContext serviceCtx = ctx.getJobletContext().getServiceContext();
INcApplicationContext appCtx = (INcApplicationContext) serviceCtx.getApplicationContext();
try {
- restoreExternalObjects(serviceCtx, appCtx.getLibraryManager());
+ restoreExternalObjects(serviceCtx, appCtx.getLibraryManager(), ctx.getWarningCollector());
} catch (Exception e) {
LOGGER.log(Level.INFO, "Failure restoring external objects", e);
throw HyracksDataException.create(e);
@@ -119,19 +121,19 @@
}
}
- private void restoreExternalObjects(IServiceContext serviceContext, ILibraryManager libraryManager)
- throws HyracksDataException, AlgebricksException {
+ private void restoreExternalObjects(IServiceContext serviceContext, ILibraryManager libraryManager,
+ IWarningCollector warningCollector) throws HyracksDataException, AlgebricksException {
if (dataSourceFactory == null) {
- dataSourceFactory = DatasourceFactoryProvider.getExternalDataSourceFactory(libraryManager, configuration);
+ dataSourceFactory = createExternalDataSourceFactory(configuration, libraryManager);
// create and configure parser factory
if (dataSourceFactory.isIndexible() && (files != null)) {
((IIndexibleExternalDataSource) dataSourceFactory).setSnapshot(files, indexingOp);
}
- dataSourceFactory.configure(serviceContext, configuration);
+ dataSourceFactory.configure(serviceContext, configuration, warningCollector);
}
if (dataParserFactory == null) {
// create and configure parser factory
- dataParserFactory = ParserFactoryProvider.getDataParserFactory(libraryManager, configuration);
+ dataParserFactory = createDataParserFactory(configuration, libraryManager);
dataParserFactory.setRecordType(recordType);
dataParserFactory.setMetaType(metaType);
dataParserFactory.configure(configuration);
@@ -139,19 +141,18 @@
}
@Override
- public void configure(IServiceContext serviceContext, Map<String, String> configuration)
- throws HyracksDataException, AlgebricksException {
+ public void configure(IServiceContext serviceContext, Map<String, String> configuration,
+ IWarningCollector warningCollector) throws HyracksDataException, AlgebricksException {
this.configuration = configuration;
IApplicationContext appCtx = (IApplicationContext) serviceContext.getApplicationContext();
ExternalDataUtils.validateDataSourceParameters(configuration);
- dataSourceFactory =
- DatasourceFactoryProvider.getExternalDataSourceFactory(appCtx.getLibraryManager(), configuration);
+ dataSourceFactory = createExternalDataSourceFactory(configuration, appCtx.getLibraryManager());
if (dataSourceFactory.isIndexible() && (files != null)) {
((IIndexibleExternalDataSource) dataSourceFactory).setSnapshot(files, indexingOp);
}
- dataSourceFactory.configure(serviceContext, configuration);
+ dataSourceFactory.configure(serviceContext, configuration, warningCollector);
ExternalDataUtils.validateDataParserParameters(configuration);
- dataParserFactory = ParserFactoryProvider.getDataParserFactory(appCtx.getLibraryManager(), configuration);
+ dataParserFactory = createDataParserFactory(configuration, appCtx.getLibraryManager());
dataParserFactory.setRecordType(recordType);
dataParserFactory.setMetaType(metaType);
dataParserFactory.configure(configuration);
@@ -222,4 +223,14 @@
dataParserFactory.configure(Collections.emptyMap());
configuration = Collections.emptyMap();
}
+
+ protected IExternalDataSourceFactory createExternalDataSourceFactory(Map<String, String> configuration,
+ ILibraryManager libraryManager) throws HyracksDataException, AsterixException {
+ return DatasourceFactoryProvider.getExternalDataSourceFactory(libraryManager, configuration);
+ }
+
+ protected IDataParserFactory createDataParserFactory(Map<String, String> configuration,
+ ILibraryManager libraryManager) throws AsterixException {
+ return ParserFactoryProvider.getDataParserFactory(libraryManager, configuration);
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/LookupAdapterFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/LookupAdapterFactory.java
index b51416a..b543804 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/LookupAdapterFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/adapter/factory/LookupAdapterFactory.java
@@ -40,6 +40,7 @@
import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
@SuppressWarnings({ "rawtypes", "unchecked" })
public class LookupAdapterFactory<T> implements Serializable {
@@ -78,15 +79,16 @@
}
}
- public void configure(IServiceContext serviceContext, Map<String, String> configuration)
- throws HyracksDataException, AlgebricksException {
+ public void configure(IServiceContext serviceContext, Map<String, String> configuration,
+ IWarningCollector warningCollector) throws HyracksDataException, AlgebricksException {
this.configuration = configuration;
IApplicationContext appCtx = (IApplicationContext) serviceContext.getApplicationContext();
- readerFactory = LookupReaderFactoryProvider.getLookupReaderFactory(serviceContext, configuration);
+ readerFactory =
+ LookupReaderFactoryProvider.getLookupReaderFactory(serviceContext, configuration, warningCollector);
dataParserFactory = (IRecordDataParserFactory<T>) ParserFactoryProvider
.getDataParserFactory(appCtx.getLibraryManager(), configuration);
dataParserFactory.setRecordType(recordType);
- readerFactory.configure(serviceContext, configuration);
+ readerFactory.configure(serviceContext, configuration, warningCollector);
dataParserFactory.configure(configuration);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/AsterixInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/AsterixInputStream.java
index a4c2fae..f959f8d 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/AsterixInputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/AsterixInputStream.java
@@ -47,4 +47,12 @@
public void setNotificationHandler(IStreamNotificationHandler notificationHandler) {
this.notificationHandler = notificationHandler;
}
+
+ public String getStreamName() {
+ return "";
+ }
+
+ public String getPreviousStreamName() {
+ return "";
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IAdapterFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IAdapterFactory.java
deleted file mode 100644
index 40bc7d8..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IAdapterFactory.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.api;
-
-import java.io.Serializable;
-import java.util.Map;
-
-import org.apache.asterix.om.types.ARecordType;
-import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.api.application.IServiceContext;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-/**
- * Base interface for IGenericDatasetAdapterFactory and ITypedDatasetAdapterFactory.
- * Acts as a marker interface indicating that the implementation provides functionality
- * for creating an adapter.
- */
-public interface IAdapterFactory extends Serializable {
-
- /**
- * Returns the display name corresponding to the Adapter type that is created by the factory.
- *
- * @return the display name
- */
- String getAlias();
-
- /**
- * Gets a list of partition constraints. A partition constraint can be a
- * requirement to execute at a particular location or could be cardinality
- * constraints indicating the number of instances that need to run in
- * parallel. example, a IDatasourceAdapter implementation written for data
- * residing on the local file system of a node cannot run on any other node
- * and thus has a location partition constraint. The location partition
- * constraint can be expressed as a node IP address or a node controller id.
- * In the former case, the IP address is translated to a node controller id
- * running on the node with the given IP address.
- *
- * @throws AlgebricksException
- */
- AlgebricksAbsolutePartitionConstraint getPartitionConstraint() throws AlgebricksException;
-
- /**
- * Creates an instance of IDatasourceAdapter.
- *
- * @param HyracksTaskContext
- * @param partition
- * @return An instance of IDatasourceAdapter.
- * @throws Exception
- */
- IDataSourceAdapter createAdapter(IHyracksTaskContext ctx, int partition) throws HyracksDataException;
-
- /**
- * Configure the adapter
- *
- * @param serviceContext
- * @param configuration
- * @throws AlgebricksException
- * @throws HyracksDataException
- */
- void configure(IServiceContext serviceContext, Map<String, String> configuration)
- throws HyracksDataException, AlgebricksException;
-
- /**
- * Set the expected record output type of the adapter
- *
- * @param outputType
- */
- void setOutputType(ARecordType outputType);
-
- /**
- * Set the expected meta output type of the adapter
- *
- * @param metaType
- */
- void setMetaType(ARecordType metaType);
-
- /**
- * @return the adapter record output type
- */
- ARecordType getOutputType();
-
- /**
- * @return the adapter meta output type
- */
- ARecordType getMetaType();
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataFlowController.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataFlowController.java
index f59b82e..ccc420b 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataFlowController.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataFlowController.java
@@ -43,4 +43,11 @@
public default boolean stop(long timeout) throws HyracksDataException {
throw new RuntimeDataException(ErrorCode.OPERATION_NOT_SUPPORTED);
}
+
+ /**
+ * @return The number of processed tuples by this controller
+ */
+ default long getProcessedTuples() {
+ throw new UnsupportedOperationException();
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataSourceAdapter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataSourceAdapter.java
deleted file mode 100644
index 472cdae..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IDataSourceAdapter.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.external.api;
-
-import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-/**
- * A super interface implemented by a data source adapter. An adapter can be a
- * pull based or push based. This interface provides all common APIs that need
- * to be implemented by each adapter irrespective of the the kind of
- * adapter(pull or push).
- */
-@FunctionalInterface
-public interface IDataSourceAdapter {
-
- public enum AdapterType {
- INTERNAL,
- EXTERNAL
- }
-
- /**
- * Triggers the adapter to begin ingesting data from the external source.
- *
- * @param partition
- * The adapter could be running with a degree of parallelism.
- * partition corresponds to the i'th parallel instance.
- * @param writer
- * The instance of frame writer that is used by the adapter to
- * write frame to. Adapter packs the fetched bytes (from external source),
- * packs them into frames and forwards the frames to an upstream receiving
- * operator using the instance of IFrameWriter.
- * @throws Exception
- */
- public void start(int partition, IFrameWriter writer) throws HyracksDataException, InterruptedException;
-}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalDataSourceFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalDataSourceFactory.java
index efa12ef..17cbff4 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalDataSourceFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IExternalDataSourceFactory.java
@@ -31,6 +31,7 @@
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
public interface IExternalDataSourceFactory extends Serializable {
@@ -63,7 +64,7 @@
* @param configuration
* @throws AsterixException
*/
- void configure(IServiceContext ctx, Map<String, String> configuration)
+ void configure(IServiceContext ctx, Map<String, String> configuration, IWarningCollector warningCollector)
throws AlgebricksException, HyracksDataException;
/**
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IIndexingAdapterFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IIndexingAdapterFactory.java
index 37cc1cf..8d42046 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IIndexingAdapterFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IIndexingAdapterFactory.java
@@ -22,6 +22,6 @@
import org.apache.asterix.external.indexing.ExternalFile;
-public interface IIndexingAdapterFactory extends IAdapterFactory {
+public interface IIndexingAdapterFactory extends ITypedAdapterFactory {
public void setSnapshot(List<ExternalFile> files, boolean indexingOp);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordConverter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordConverter.java
index 9d9ff28..f544ca0 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordConverter.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordConverter.java
@@ -19,9 +19,18 @@
package org.apache.asterix.external.api;
import java.io.IOException;
+import java.util.function.LongSupplier;
@FunctionalInterface
public interface IRecordConverter<I, O> {
public O convert(IRawRecord<? extends I> input) throws IOException;
+
+ /**
+ * Configures the converter with information suppliers from the {@link IRecordReader} data source.
+ *
+ * @param lineNumber line number supplier
+ */
+ default void configure(LongSupplier lineNumber) {
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordDataParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordDataParser.java
index be17744..c4dfdd0 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordDataParser.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordDataParser.java
@@ -19,15 +19,33 @@
package org.apache.asterix.external.api;
import java.io.DataOutput;
+import java.util.function.LongSupplier;
+import java.util.function.Supplier;
import org.apache.hyracks.api.exceptions.HyracksDataException;
public interface IRecordDataParser<T> extends IDataParser {
/**
- * @param record
- * @param out
- * @throws Exception
+ * Parses the input record and writes the result into the {@code out}. Implementations should only write to the
+ * {@code out} if the record is parsed successfully. If parsing fails, the {@code out} should never be touched. In
+ * other words, no partial writing in case of failure. Additionally, implementations may choose to issue a
+ * warning and/or throw an exception in case of failure.
+ *
+ * @param record input record to parse
+ * @param out output where the parsed record is written into
+ *
+ * @return true if the record was parsed successfully and written to out. False, otherwise.
+ * @throws HyracksDataException HyracksDataException
*/
- public void parse(IRawRecord<? extends T> record, DataOutput out) throws HyracksDataException;
+ public boolean parse(IRawRecord<? extends T> record, DataOutput out) throws HyracksDataException;
+
+ /**
+ * Configures the parser with information suppliers from the {@link IRecordReader} data source.
+ *
+ * @param dataSourceName data source name supplier
+ * @param lineNumber line number supplier
+ */
+ default void configure(Supplier<String> dataSourceName, LongSupplier lineNumber) {
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReader.java
index 9033814..cb97526 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReader.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/IRecordReader.java
@@ -20,8 +20,11 @@
import java.io.Closeable;
import java.io.IOException;
+import java.util.function.LongSupplier;
+import java.util.function.Supplier;
import org.apache.asterix.external.dataflow.AbstractFeedDataFlowController;
+import org.apache.asterix.external.util.ExternalDataConstants;
import org.apache.asterix.external.util.FeedLogManager;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -84,4 +87,12 @@
default String getStats() {
return null;
}
+
+ default Supplier<String> getDataSourceName() {
+ return ExternalDataConstants.EMPTY_STRING;
+ }
+
+ default LongSupplier getLineNumber() {
+ return ExternalDataConstants.NO_LINES;
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/ITypedAdapterFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/ITypedAdapterFactory.java
new file mode 100644
index 0000000..13e3b34
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/api/ITypedAdapterFactory.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.api;
+
+import org.apache.asterix.common.external.IAdapterFactory;
+import org.apache.asterix.om.types.ARecordType;
+
+/**
+ * Base interface for IGenericDatasetAdapterFactory and ITypedDatasetAdapterFactory.
+ * Acts as a marker interface indicating that the implementation provides functionality
+ * for creating an adapter.
+ */
+public interface ITypedAdapterFactory extends IAdapterFactory {
+
+ /**
+ * Set the expected record output type of the adapter
+ *
+ * @param outputType
+ */
+ void setOutputType(ARecordType outputType);
+
+ /**
+ * Set the expected meta output type of the adapter
+ *
+ * @param metaType
+ */
+ void setMetaType(ARecordType metaType);
+
+ /**
+ * @return the adapter record output type
+ */
+ ARecordType getOutputType();
+
+ /**
+ * @return the adapter meta output type
+ */
+ ARecordType getMetaType();
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/AbstractDataFlowController.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/AbstractDataFlowController.java
index a324496..95024e1 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/AbstractDataFlowController.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/AbstractDataFlowController.java
@@ -24,8 +24,14 @@
public abstract class AbstractDataFlowController implements IDataFlowController {
protected final IHyracksTaskContext ctx;
+ protected long processedTuples = 0;
public AbstractDataFlowController(IHyracksTaskContext ctx) {
this.ctx = ctx;
}
+
+ @Override
+ public long getProcessedTuples() {
+ return processedTuples;
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
index f392139..947257c 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
@@ -93,8 +93,7 @@
}
} catch (HyracksDataException e) {
LOGGER.log(Level.WARN, "Exception during ingestion", e);
- if (e.getComponent() == ErrorCode.ASTERIX
- && (e.getErrorCode() == ErrorCode.FEED_FAILED_WHILE_GETTING_A_NEW_RECORD)) {
+ if (e.matches(ErrorCode.FEED_FAILED_WHILE_GETTING_A_NEW_RECORD)) {
// Failure but we know we can for sure push the previously parsed records safely
failure = e;
try {
@@ -177,14 +176,19 @@
}
private boolean parseAndForward(IRawRecord<? extends T> record) throws IOException {
+ boolean success;
try {
- dataParser.parse(record, tb.getDataOutput());
+ success = dataParser.parse(record, tb.getDataOutput());
} catch (Exception e) {
LOGGER.log(Level.WARN, ExternalDataConstants.ERROR_PARSE_RECORD, e);
feedLogManager.logRecord(record.toString(), ExternalDataConstants.ERROR_PARSE_RECORD);
// continue the outer loop
return false;
}
+ if (!success) {
+ // continue the outer loop
+ return false;
+ }
tb.addFieldEndOffset();
addMetaPart(tb, record);
addPrimaryKeys(tb, record);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/RecordDataFlowController.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/RecordDataFlowController.java
index c028965..2c19f9d 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/RecordDataFlowController.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/RecordDataFlowController.java
@@ -44,15 +44,18 @@
@Override
public void start(IFrameWriter writer) throws HyracksDataException {
try {
+ processedTuples = 0;
ArrayTupleBuilder tb = new ArrayTupleBuilder(numOfTupleFields);
TupleForwarder tupleForwarder = new TupleForwarder(ctx, writer);
while (recordReader.hasNext()) {
IRawRecord<? extends T> record = recordReader.next();
tb.reset();
- dataParser.parse(record, tb.getDataOutput());
- tb.addFieldEndOffset();
- appendOtherTupleFields(tb);
- tupleForwarder.addTuple(tb);
+ if (dataParser.parse(record, tb.getDataOutput())) {
+ tb.addFieldEndOffset();
+ appendOtherTupleFields(tb);
+ tupleForwarder.addTuple(tb);
+ processedTuples++;
+ }
}
tupleForwarder.complete();
} catch (Exception e) {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/StreamDataFlowController.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/StreamDataFlowController.java
index 8275953..9c11c97 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/StreamDataFlowController.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/StreamDataFlowController.java
@@ -35,6 +35,7 @@
@Override
public void start(IFrameWriter writer) throws HyracksDataException {
try {
+ processedTuples = 0;
ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
TupleForwarder tupleForwarder = new TupleForwarder(ctx, writer);
while (true) {
@@ -44,6 +45,7 @@
}
tb.addFieldEndOffset();
tupleForwarder.addTuple(tb);
+ processedTuples++;
}
tupleForwarder.complete();
} catch (Exception e) {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/FeedAdapter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/FeedAdapter.java
index 2a92d40..123a552 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/FeedAdapter.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/FeedAdapter.java
@@ -21,7 +21,7 @@
import java.io.Closeable;
import java.io.IOException;
-import org.apache.asterix.external.api.IDataSourceAdapter;
+import org.apache.asterix.common.external.IDataSourceAdapter;
import org.apache.asterix.external.dataflow.AbstractFeedDataFlowController;
import org.apache.hyracks.api.comm.IFrameWriter;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -38,6 +38,11 @@
controller.start(writer);
}
+ @Override
+ public long getProcessedTuples() {
+ return controller.getProcessedTuples();
+ }
+
public boolean stop(long timeout) throws HyracksDataException {
return controller.stop(timeout);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/GenericAdapter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/GenericAdapter.java
index 916fe0a..17a134b 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/GenericAdapter.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/GenericAdapter.java
@@ -18,8 +18,8 @@
*/
package org.apache.asterix.external.dataset.adapter;
+import org.apache.asterix.common.external.IDataSourceAdapter;
import org.apache.asterix.external.api.IDataFlowController;
-import org.apache.asterix.external.api.IDataSourceAdapter;
import org.apache.hyracks.api.comm.IFrameWriter;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -35,4 +35,9 @@
public void start(int partition, IFrameWriter writer) throws HyracksDataException, InterruptedException {
controller.start(writer);
}
+
+ @Override
+ public long getProcessedTuples() {
+ return controller.getProcessedTuples();
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/LookupAdapter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/LookupAdapter.java
index 82e9bba..5700f9d 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/LookupAdapter.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataset/adapter/LookupAdapter.java
@@ -127,9 +127,10 @@
propagateInputFields(tupleIndex);
}
if (record != null) {
- dataParser.parse(record, tb.getDataOutput());
- tb.addFieldEndOffset();
- DataflowUtils.addTupleToFrame(appender, tb, writer);
+ if (dataParser.parse(record, tb.getDataOutput())) {
+ tb.addFieldEndOffset();
+ DataflowUtils.addTupleToFrame(appender, tb, writer);
+ }
} else if (retainNull) {
tb.getDataOutput().write(missingTupleBuild.getByteArray());
tb.addFieldEndOffset();
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedExceptionHandler.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedExceptionHandler.java
index 2fda99f..a836ef9 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedExceptionHandler.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/dataflow/FeedExceptionHandler.java
@@ -46,7 +46,7 @@
@Override
public ByteBuffer handle(HyracksDataException th, ByteBuffer frame) {
try {
- if (th.getErrorCode() == ErrorCode.ERROR_PROCESSING_TUPLE) {
+ if (th.matches(ErrorCode.ERROR_PROCESSING_TUPLE)) {
// TODO(amoudi): add check for cause. cause should be either cast or duplicate key
fta.reset(frame);
int tupleIndex = (int) (th.getParams()[0]);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/AbstractSubscriber.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/AbstractSubscriber.java
index 37b157e..726880d 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/AbstractSubscriber.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/AbstractSubscriber.java
@@ -18,6 +18,8 @@
*/
package org.apache.asterix.external.feed.watch;
+import java.util.Objects;
+
import org.apache.asterix.active.IActiveEntityEventSubscriber;
import org.apache.asterix.active.IActiveEntityEventsListener;
import org.apache.hyracks.util.Span;
@@ -25,11 +27,19 @@
public abstract class AbstractSubscriber implements IActiveEntityEventSubscriber {
protected final IActiveEntityEventsListener listener;
+ private final Object lockObject;
private volatile boolean done = false;
private volatile Exception failure = null;
+ public AbstractSubscriber(IActiveEntityEventsListener listener, Object lockObject) {
+ Objects.requireNonNull(lockObject);
+ this.listener = listener;
+ this.lockObject = lockObject;
+ }
+
public AbstractSubscriber(IActiveEntityEventsListener listener) {
this.listener = listener;
+ this.lockObject = this;
}
@Override
@@ -38,28 +48,28 @@
}
public void complete(Exception failure) {
- synchronized (listener) {
+ synchronized (lockObject) {
if (failure != null) {
this.failure = failure;
}
done = true;
- listener.notifyAll();
+ lockObject.notifyAll();
}
}
@Override
public void sync() throws InterruptedException {
- synchronized (listener) {
+ synchronized (lockObject) {
while (!done) {
- listener.wait();
+ lockObject.wait();
}
}
}
public boolean sync(Span span) throws InterruptedException {
- synchronized (listener) {
+ synchronized (lockObject) {
while (!done) {
- span.wait(listener);
+ span.wait(lockObject);
if (done || span.elapsed()) {
return done;
}
@@ -71,4 +81,11 @@
public Exception getFailure() {
return failure;
}
+
+ protected void reset() {
+ synchronized (lockObject) {
+ done = false;
+ failure = null;
+ }
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/WaitForStateSubscriber.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/WaitForStateSubscriber.java
index 818d826..4dc86ac 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/WaitForStateSubscriber.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/feed/watch/WaitForStateSubscriber.java
@@ -29,7 +29,7 @@
private final Set<ActivityState> targetStates;
public WaitForStateSubscriber(IActiveEntityEventsListener listener, Set<ActivityState> targetStates) {
- super(listener);
+ super(listener, listener);
this.targetStates = targetStates;
listener.subscribe(this);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
index f830376..dba0f27 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/HDFSDataSourceFactory.java
@@ -51,6 +51,7 @@
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
import org.apache.hyracks.hdfs.dataflow.ConfFactory;
import org.apache.hyracks.hdfs.dataflow.InputSplitsFactory;
import org.apache.hyracks.hdfs.scheduler.Scheduler;
@@ -80,7 +81,8 @@
private static final List<String> recordReaderNames = Collections.unmodifiableList(Arrays.asList("hdfs"));
@Override
- public void configure(IServiceContext serviceCtx, Map<String, String> configuration) throws AsterixException {
+ public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
+ IWarningCollector warningCollector) throws AsterixException {
try {
this.serviceCtx = serviceCtx;
this.configuration = configuration;
@@ -205,7 +207,7 @@
IExternalIndexer indexer = files == null ? null : ExternalIndexerProvider.getIndexer(configuration);
if (recordReaderClazz != null) {
StreamRecordReader streamReader = (StreamRecordReader) recordReaderClazz.getConstructor().newInstance();
- streamReader.configure(createInputStream(ctx, partition, indexer), configuration);
+ streamReader.configure(ctx, createInputStream(ctx, partition, indexer), configuration);
if (indexer != null) {
return new IndexingStreamRecordReader(streamReader, indexer);
} else {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/CharArrayRecord.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/CharArrayRecord.java
index 65ecd8d..5d7ffb2 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/CharArrayRecord.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/CharArrayRecord.java
@@ -65,7 +65,7 @@
size = length;
}
- private void ensureCapacity(int len) throws IOException {
+ private void ensureCapacity(int len) throws RuntimeDataException {
if (value.length < len) {
if (len > ExternalDataConstants.MAX_RECORD_SIZE) {
throw new RuntimeDataException(ErrorCode.INPUT_RECORD_READER_CHAR_ARRAY_RECORD_TOO_LARGE,
@@ -77,7 +77,7 @@
}
}
- public void append(char[] recordBuffer, int offset, int length) throws IOException {
+ public void append(char[] recordBuffer, int offset, int length) throws RuntimeDataException {
ensureCapacity(size + length);
System.arraycopy(recordBuffer, offset, value, size, length);
size += length;
@@ -134,4 +134,10 @@
strValue.getChars(0, strValue.length(), value, 0);
this.size = strValue.length();
}
+
+ public boolean isEmptyRecord() {
+ return size <= 0
+ || (size == 1 && (value[0] == ExternalDataConstants.LF || value[0] == ExternalDataConstants.CR))
+ || (size == 2 && value[0] == ExternalDataConstants.CR && value[1] == ExternalDataConstants.LF);
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/RecordWithMetadataAndPK.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/RecordWithMetadataAndPK.java
index 959e34d..de889a2 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/RecordWithMetadataAndPK.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/RecordWithMetadataAndPK.java
@@ -155,7 +155,9 @@
fieldValueBufferOutputs[index].writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
} else {
fieldValueBufferOutputs[index].writeByte(fieldTypeTags[index]);
- valueParsers[index].parse(src, offset, length, fieldValueBufferOutputs[index]);
+ if (!valueParsers[index].parse(src, offset, length, fieldValueBufferOutputs[index])) {
+ throw new RuntimeDataException(ErrorCode.FAILED_TO_PARSE_METADATA);
+ }
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/converter/CSVToRecordWithMetadataAndPKConverter.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/converter/CSVToRecordWithMetadataAndPKConverter.java
index 8255ebb..8b930aa 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/converter/CSVToRecordWithMetadataAndPKConverter.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/converter/CSVToRecordWithMetadataAndPKConverter.java
@@ -19,13 +19,17 @@
package org.apache.asterix.external.input.record.converter;
import java.io.IOException;
+import java.util.function.LongSupplier;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.external.api.IRawRecord;
import org.apache.asterix.external.input.record.CharArrayRecord;
import org.apache.asterix.external.input.record.RecordWithMetadataAndPK;
import org.apache.asterix.external.util.ExternalDataConstants;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
import org.apache.hyracks.dataflow.std.file.FieldCursorForDelimitedDataParser;
public class CSVToRecordWithMetadataAndPKConverter
@@ -35,13 +39,16 @@
private final int valueIndex;
private final RecordWithMetadataAndPK<char[]> recordWithMetadata;
private final CharArrayRecord record;
+ private LongSupplier lineNumber = ExternalDataConstants.NO_LINES;
public CSVToRecordWithMetadataAndPKConverter(final int valueIndex, final char delimiter, final ARecordType metaType,
- final ARecordType recordType, final int[] keyIndicator, final int[] keyIndexes, final IAType[] keyTypes) {
- this.cursor = new FieldCursorForDelimitedDataParser(null, delimiter, ExternalDataConstants.QUOTE);
+ final ARecordType recordType, final int[] keyIndicator, final int[] keyIndexes, final IAType[] keyTypes,
+ IWarningCollector warningCollector) {
+ this.cursor = new FieldCursorForDelimitedDataParser(null, delimiter, ExternalDataConstants.QUOTE,
+ warningCollector, ExternalDataConstants.EMPTY_STRING);
this.record = new CharArrayRecord();
this.valueIndex = valueIndex;
- this.recordWithMetadata = new RecordWithMetadataAndPK<char[]>(record, metaType.getFieldTypes(), recordType,
+ this.recordWithMetadata = new RecordWithMetadataAndPK<>(record, metaType.getFieldTypes(), recordType,
keyIndicator, keyIndexes, keyTypes);
}
@@ -49,24 +56,32 @@
public RecordWithMetadataAndPK<char[]> convert(final IRawRecord<? extends char[]> input) throws IOException {
record.reset();
recordWithMetadata.reset();
- cursor.nextRecord(input.get(), input.size());
+ cursor.nextRecord(input.get(), input.size(), lineNumber.getAsLong());
int i = 0;
int j = 0;
- while (cursor.nextField()) {
- if (cursor.isDoubleQuoteIncludedInThisField) {
- cursor.eliminateDoubleQuote(cursor.buffer, cursor.fStart, cursor.fEnd - cursor.fStart);
- cursor.fEnd -= cursor.doubleQuoteCount;
- cursor.isDoubleQuoteIncludedInThisField = false;
+ FieldCursorForDelimitedDataParser.Result lastResult;
+ while ((lastResult = cursor.nextField()) == FieldCursorForDelimitedDataParser.Result.OK) {
+ if (cursor.fieldHasDoubleQuote()) {
+ cursor.eliminateDoubleQuote();
}
if (i == valueIndex) {
- record.setValue(cursor.buffer, cursor.fStart, cursor.fEnd - cursor.fStart);
+ record.setValue(cursor.getBuffer(), cursor.getFieldStart(), cursor.getFieldLength());
record.endRecord();
} else {
- recordWithMetadata.setRawMetadata(j, cursor.buffer, cursor.fStart, cursor.fEnd - cursor.fStart);
+ recordWithMetadata.setRawMetadata(j, cursor.getBuffer(), cursor.getFieldStart(),
+ cursor.getFieldLength());
j++;
}
i++;
}
+ if (lastResult == FieldCursorForDelimitedDataParser.Result.ERROR) {
+ throw new RuntimeDataException(ErrorCode.FAILED_TO_PARSE_RECORD);
+ }
return recordWithMetadata;
}
+
+ @Override
+ public void configure(LongSupplier lineNumber) {
+ this.lineNumber = lineNumber == null ? ExternalDataConstants.NO_LINES : lineNumber;
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/converter/CSVWithRecordConverterFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/converter/CSVWithRecordConverterFactory.java
index ee16228..a9e5bc7 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/converter/CSVWithRecordConverterFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/converter/CSVWithRecordConverterFactory.java
@@ -27,6 +27,7 @@
import org.apache.asterix.external.util.ExternalDataConstants;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
public class CSVWithRecordConverterFactory implements IRecordConverterFactory<char[], RecordWithMetadataAndPK<char[]>> {
@@ -40,9 +41,9 @@
private IAType[] keyTypes;
@Override
- public IRecordConverter<char[], RecordWithMetadataAndPK<char[]>> createConverter() {
+ public IRecordConverter<char[], RecordWithMetadataAndPK<char[]>> createConverter(IHyracksTaskContext ctx) {
return new CSVToRecordWithMetadataAndPKConverter(recordIndex, delimiter, metaType, recordType, keyIndicators,
- keyIndexes, keyTypes);
+ keyIndexes, keyTypes, ctx.getWarningCollector());
}
@Override
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/converter/DCPConverterFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/converter/DCPConverterFactory.java
index dc93533..b228b94 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/converter/DCPConverterFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/converter/DCPConverterFactory.java
@@ -24,6 +24,7 @@
import org.apache.asterix.external.api.IRecordConverter;
import org.apache.asterix.external.input.record.RecordWithMetadataAndPK;
import org.apache.asterix.om.types.ARecordType;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
import com.couchbase.client.core.message.dcp.DCPRequest;
@@ -50,7 +51,7 @@
}
@Override
- public IRecordConverter<DCPRequest, RecordWithMetadataAndPK<char[]>> createConverter() {
+ public IRecordConverter<DCPRequest, RecordWithMetadataAndPK<char[]>> createConverter(IHyracksTaskContext ctx) {
return new DCPMessageToRecordConverter();
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/converter/IRecordConverterFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/converter/IRecordConverterFactory.java
index 4990527..875a331 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/converter/IRecordConverterFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/converter/IRecordConverterFactory.java
@@ -24,10 +24,11 @@
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.external.api.IRecordConverter;
import org.apache.asterix.om.types.ARecordType;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
public interface IRecordConverterFactory<I, O> extends Serializable {
- public IRecordConverter<I, O> createConverter();
+ public IRecordConverter<I, O> createConverter(IHyracksTaskContext ctx);
public void configure(Map<String, String> configuration) throws AsterixException;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java
new file mode 100644
index 0000000..4d5288c
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStream.java
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader.aws;
+
+import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.zip.GZIPInputStream;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.external.input.stream.AbstractMultipleInputStream;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.CleanupUtils;
+import org.apache.hyracks.util.LogRedactionUtil;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import software.amazon.awssdk.core.exception.SdkException;
+import software.amazon.awssdk.services.s3.S3Client;
+import software.amazon.awssdk.services.s3.model.GetObjectRequest;
+import software.amazon.awssdk.services.s3.model.NoSuchKeyException;
+import software.amazon.awssdk.services.s3.model.S3Exception;
+
+public class AwsS3InputStream extends AbstractMultipleInputStream {
+
+ private static final Logger LOGGER = LogManager.getLogger();
+
+ // Configuration
+ private final String bucket;
+ private final int bufferSize;
+
+ private final S3Client s3Client;
+ private static final int MAX_RETRIES = 5; // We will retry 5 times in case of internal error from AWS S3 service
+
+ // File fields
+ private final List<String> filePaths;
+ private int nextFileIndex = 0;
+
+ public AwsS3InputStream(Map<String, String> configuration, List<String> filePaths) throws HyracksDataException {
+ this.filePaths = filePaths;
+ this.s3Client = buildAwsS3Client(configuration);
+ this.bufferSize = ExternalDataUtils.getOrDefaultBufferSize(configuration);
+ this.bucket = configuration.get(AwsS3.CONTAINER_NAME_FIELD_NAME);
+ }
+
+ @Override
+ protected boolean advance() throws IOException {
+ // No files to read for this partition
+ if (filePaths == null || filePaths.isEmpty()) {
+ return false;
+ }
+
+ // Finished reading all the files
+ if (nextFileIndex >= filePaths.size()) {
+ return false;
+ }
+
+ // Close the current stream before going to the next one
+ if (in != null) {
+ CleanupUtils.close(in, null);
+ }
+
+ String fileName = filePaths.get(nextFileIndex);
+ GetObjectRequest.Builder getObjectBuilder = GetObjectRequest.builder();
+ GetObjectRequest getObjectRequest = getObjectBuilder.bucket(bucket).key(fileName).build();
+
+ boolean isAvailableStream = doGetInputStream(getObjectRequest);
+ nextFileIndex++;
+ if (!isAvailableStream) {
+ return advance();
+ }
+
+ // Use gzip stream if needed
+ if (StringUtils.endsWithIgnoreCase(fileName, ".gz") || StringUtils.endsWithIgnoreCase(fileName, ".gzip")) {
+ in = new GZIPInputStream(in, bufferSize);
+ }
+
+ // Current file ready, point to the next file
+ if (notificationHandler != null) {
+ notificationHandler.notifyNewSource();
+ }
+ return true;
+ }
+
+ /**
+ * Get the input stream. If an error is encountered, depending on the error code, a retry might be favorable.
+ *
+ * @return true
+ */
+ private boolean doGetInputStream(GetObjectRequest request) throws RuntimeDataException {
+ int retries = 0;
+ while (retries < MAX_RETRIES) {
+ try {
+ in = s3Client.getObject(request);
+ break;
+ } catch (NoSuchKeyException ex) {
+ LOGGER.debug(() -> "Key " + LogRedactionUtil.userData(request.key()) + " was not found in bucket "
+ + request.bucket());
+ return false;
+ } catch (S3Exception ex) {
+ if (!shouldRetry(ex.awsErrorDetails().errorCode(), retries++)) {
+ throw new RuntimeDataException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex.getMessage());
+ }
+ LOGGER.debug(() -> "S3 retryable error: " + LogRedactionUtil.userData(ex.getMessage()));
+
+ // Backoff for 1 sec for the first 2 retries, and 2 seconds from there onward
+ try {
+ Thread.sleep(TimeUnit.SECONDS.toMillis(retries < 3 ? 1 : 2));
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+ } catch (SdkException ex) {
+ throw new RuntimeDataException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex.getMessage());
+ }
+ }
+ return true;
+ }
+
+ private boolean shouldRetry(String errorCode, int currentRetry) {
+ return currentRetry < MAX_RETRIES && AwsS3.isRetryableError(errorCode);
+ }
+
+ private S3Client buildAwsS3Client(Map<String, String> configuration) throws HyracksDataException {
+ try {
+ return ExternalDataUtils.AwsS3.buildAwsS3Client(configuration);
+ } catch (CompilationException ex) {
+ throw HyracksDataException.create(ex);
+ }
+ }
+
+ @Override
+ public boolean stop() {
+ return false;
+ }
+
+ @Override
+ public boolean handleException(Throwable th) {
+ return false;
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (in != null) {
+ CleanupUtils.close(in, null);
+ }
+ if (s3Client != null) {
+ CleanupUtils.close(s3Client, null);
+ }
+ }
+
+ @Override
+ public String getStreamName() {
+ return getStreamNameAt(nextFileIndex - 1);
+ }
+
+ @Override
+ public String getPreviousStreamName() {
+ return getStreamNameAt(nextFileIndex - 2);
+ }
+
+ private String getStreamNameAt(int fileIndex) {
+ return fileIndex < 0 || filePaths == null || filePaths.isEmpty() ? "" : filePaths.get(fileIndex);
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java
new file mode 100644
index 0000000..715c5df
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3InputStreamFactory.java
@@ -0,0 +1,341 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader.aws;
+
+import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_EXCLUDE;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_INCLUDE;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiPredicate;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.regex.PatternSyntaxException;
+
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.WarningUtil;
+import org.apache.asterix.external.api.AsterixInputStream;
+import org.apache.asterix.external.api.IInputStreamFactory;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.application.IServiceContext;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.Warning;
+import org.apache.hyracks.api.util.CleanupUtils;
+
+import software.amazon.awssdk.core.exception.SdkException;
+import software.amazon.awssdk.services.s3.S3Client;
+import software.amazon.awssdk.services.s3.model.ListObjectsRequest;
+import software.amazon.awssdk.services.s3.model.ListObjectsResponse;
+import software.amazon.awssdk.services.s3.model.ListObjectsV2Request;
+import software.amazon.awssdk.services.s3.model.ListObjectsV2Response;
+import software.amazon.awssdk.services.s3.model.S3Exception;
+import software.amazon.awssdk.services.s3.model.S3Object;
+
+public class AwsS3InputStreamFactory implements IInputStreamFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private Map<String, String> configuration;
+ private final List<PartitionWorkLoadBasedOnSize> partitionWorkLoadsBasedOnSize = new ArrayList<>();
+ private transient AlgebricksAbsolutePartitionConstraint partitionConstraint;
+
+ @Override
+ public DataSourceType getDataSourceType() {
+ return DataSourceType.STREAM;
+ }
+
+ @Override
+ public boolean isIndexible() {
+ return false;
+ }
+
+ @Override
+ public AsterixInputStream createInputStream(IHyracksTaskContext ctx, int partition) throws HyracksDataException {
+ return new AwsS3InputStream(configuration, partitionWorkLoadsBasedOnSize.get(partition).getFilePaths());
+ }
+
+ @Override
+ public AlgebricksAbsolutePartitionConstraint getPartitionConstraint() {
+ return partitionConstraint;
+ }
+
+ @Override
+ public void configure(IServiceContext ctx, Map<String, String> configuration, IWarningCollector warningCollector)
+ throws AlgebricksException {
+ this.configuration = configuration;
+ ICcApplicationContext ccApplicationContext = (ICcApplicationContext) ctx.getApplicationContext();
+
+ // Ensure the validity of include/exclude
+ ExternalDataUtils.AwsS3.validateIncludeExclude(configuration);
+
+ // Get and compile the patterns for include/exclude if provided
+ List<Matcher> includeMatchers = new ArrayList<>();
+ List<Matcher> excludeMatchers = new ArrayList<>();
+ String pattern = null;
+ try {
+ for (Map.Entry<String, String> entry : configuration.entrySet()) {
+ if (entry.getKey().startsWith(KEY_INCLUDE)) {
+ pattern = entry.getValue();
+ includeMatchers.add(Pattern.compile(ExternalDataUtils.patternToRegex(pattern)).matcher(""));
+ } else if (entry.getKey().startsWith(KEY_EXCLUDE)) {
+ pattern = entry.getValue();
+ excludeMatchers.add(Pattern.compile(ExternalDataUtils.patternToRegex(pattern)).matcher(""));
+ }
+ }
+ } catch (PatternSyntaxException ex) {
+ throw new CompilationException(ErrorCode.INVALID_REGEX_PATTERN, pattern);
+ }
+
+ List<Matcher> matchersList;
+ BiPredicate<List<Matcher>, String> p;
+ if (!includeMatchers.isEmpty()) {
+ matchersList = includeMatchers;
+ p = (matchers, key) -> ExternalDataUtils.matchPatterns(matchers, key);
+ } else if (!excludeMatchers.isEmpty()) {
+ matchersList = excludeMatchers;
+ p = (matchers, key) -> !ExternalDataUtils.matchPatterns(matchers, key);
+ } else {
+ matchersList = Collections.emptyList();
+ p = (matchers, key) -> true;
+ }
+
+ // Get all objects in a bucket and extract the paths to files
+ List<S3Object> filesOnly;
+ String container = configuration.get(AwsS3.CONTAINER_NAME_FIELD_NAME);
+ S3Client s3Client = ExternalDataUtils.AwsS3.buildAwsS3Client(configuration);
+
+ try {
+ filesOnly = listS3Objects(s3Client, container, matchersList, p);
+ } catch (S3Exception ex) {
+ // New API is not implemented, try falling back to old API
+ try {
+ // For error code, see https://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html
+ if (ex.awsErrorDetails().errorCode().equals(AwsS3.ERROR_METHOD_NOT_IMPLEMENTED)) {
+ filesOnly = oldApiListS3Objects(s3Client, container, matchersList, p);
+ } else {
+ throw ex;
+ }
+ } catch (SdkException ex2) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex2.getMessage());
+ }
+ } catch (SdkException ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex.getMessage());
+ } finally {
+ if (s3Client != null) {
+ CleanupUtils.close(s3Client, null);
+ }
+ }
+
+ // Warn if no files are returned
+ if (filesOnly.isEmpty() && warningCollector.shouldWarn()) {
+ Warning warning = WarningUtil.forAsterix(null, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+ warningCollector.warn(warning);
+ }
+
+ // Partition constraints
+ partitionConstraint = ccApplicationContext.getClusterStateManager().getClusterLocations();
+ int partitionsCount = partitionConstraint.getLocations().length;
+
+ // Distribute work load amongst the partitions
+ distributeWorkLoad(filesOnly, partitionsCount);
+ }
+
+ /**
+ * Uses the latest API to retrieve the objects from the storage.
+ *
+ * @param s3Client S3 client
+ * @param container container name
+ * @param matchersList include/exclude matchers to apply
+ * @param predicate predicate to use for comparison
+ */
+ private List<S3Object> listS3Objects(S3Client s3Client, String container, List<Matcher> matchersList,
+ BiPredicate<List<Matcher>, String> predicate) {
+ String newMarker = null;
+ List<S3Object> filesOnly = new ArrayList<>();
+
+ ListObjectsV2Response listObjectsResponse;
+ ListObjectsV2Request.Builder listObjectsBuilder = ListObjectsV2Request.builder().bucket(container);
+ listObjectsBuilder.prefix(ExternalDataUtils.getPrefix(configuration));
+
+ while (true) {
+ // List the objects from the start, or from the last marker in case of truncated result
+ if (newMarker == null) {
+ listObjectsResponse = s3Client.listObjectsV2(listObjectsBuilder.build());
+ } else {
+ listObjectsResponse = s3Client.listObjectsV2(listObjectsBuilder.continuationToken(newMarker).build());
+ }
+
+ // Collect the paths to files only
+ collectAndFilterFiles(listObjectsResponse.contents(), predicate, matchersList, filesOnly);
+
+ // Mark the flag as done if done, otherwise, get the marker of the previous response for the next request
+ if (!listObjectsResponse.isTruncated()) {
+ break;
+ } else {
+ newMarker = listObjectsResponse.nextContinuationToken();
+ }
+ }
+
+ return filesOnly;
+ }
+
+ /**
+ * Uses the old API (in case the new API is not implemented) to retrieve the objects from the storage
+ *
+ * @param s3Client S3 client
+ * @param container container name
+ * @param matchersList include/exclude matchers to apply
+ * @param predicate predicate to use for comparison
+ */
+ private List<S3Object> oldApiListS3Objects(S3Client s3Client, String container, List<Matcher> matchersList,
+ BiPredicate<List<Matcher>, String> predicate) {
+ String newMarker = null;
+ List<S3Object> filesOnly = new ArrayList<>();
+
+ ListObjectsResponse listObjectsResponse;
+ ListObjectsRequest.Builder listObjectsBuilder = ListObjectsRequest.builder().bucket(container);
+ listObjectsBuilder.prefix(ExternalDataUtils.getPrefix(configuration));
+
+ while (true) {
+ // List the objects from the start, or from the last marker in case of truncated result
+ if (newMarker == null) {
+ listObjectsResponse = s3Client.listObjects(listObjectsBuilder.build());
+ } else {
+ listObjectsResponse = s3Client.listObjects(listObjectsBuilder.marker(newMarker).build());
+ }
+
+ // Collect the paths to files only
+ collectAndFilterFiles(listObjectsResponse.contents(), predicate, matchersList, filesOnly);
+
+ // Mark the flag as done if done, otherwise, get the marker of the previous response for the next request
+ if (!listObjectsResponse.isTruncated()) {
+ break;
+ } else {
+ newMarker = listObjectsResponse.nextMarker();
+ }
+ }
+
+ return filesOnly;
+ }
+
+ /**
+ * AWS S3 returns all the objects as paths, not differentiating between folder and files. The path is considered
+ * a file if it does not end up with a "/" which is the separator in a folder structure.
+ *
+ * @param s3Objects List of returned objects
+ */
+ private void collectAndFilterFiles(List<S3Object> s3Objects, BiPredicate<List<Matcher>, String> predicate,
+ List<Matcher> matchers, List<S3Object> filesOnly) {
+ for (S3Object object : s3Objects) {
+ // skip folders
+ if (object.key().endsWith("/")) {
+ continue;
+ }
+
+ // No filter, add file
+ if (predicate.test(matchers, object.key())) {
+ filesOnly.add(object);
+ }
+ }
+ }
+
+ /**
+ * To efficiently utilize the parallelism, work load will be distributed amongst the partitions based on the file
+ * size.
+ *
+ * Example:
+ * File1 1mb, File2 300kb, File3 300kb, File4 300kb
+ *
+ * Distribution:
+ * Partition1: [File1]
+ * Partition2: [File2, File3, File4]
+ *
+ * @param fileObjects AWS S3 file objects
+ * @param partitionsCount Partitions count
+ */
+ private void distributeWorkLoad(List<S3Object> fileObjects, int partitionsCount) {
+ // Prepare the workloads based on the number of partitions
+ for (int i = 0; i < partitionsCount; i++) {
+ partitionWorkLoadsBasedOnSize.add(new PartitionWorkLoadBasedOnSize());
+ }
+
+ for (S3Object object : fileObjects) {
+ PartitionWorkLoadBasedOnSize smallest = getSmallestWorkLoad();
+ smallest.addFilePath(object.key(), object.size());
+ }
+ }
+
+ /**
+ * Finds the smallest workload and returns it
+ *
+ * @return the smallest workload
+ */
+ private PartitionWorkLoadBasedOnSize getSmallestWorkLoad() {
+ PartitionWorkLoadBasedOnSize smallest = partitionWorkLoadsBasedOnSize.get(0);
+ for (PartitionWorkLoadBasedOnSize partition : partitionWorkLoadsBasedOnSize) {
+ // If the current total size is 0, add the file directly as this is a first time partition
+ if (partition.getTotalSize() == 0) {
+ smallest = partition;
+ break;
+ }
+ if (partition.getTotalSize() < smallest.getTotalSize()) {
+ smallest = partition;
+ }
+ }
+
+ return smallest;
+ }
+
+ public static class PartitionWorkLoadBasedOnSize implements Serializable {
+ private static final long serialVersionUID = 1L;
+ private final List<String> filePaths = new ArrayList<>();
+ private long totalSize = 0;
+
+ PartitionWorkLoadBasedOnSize() {
+ }
+
+ public List<String> getFilePaths() {
+ return filePaths;
+ }
+
+ public void addFilePath(String filePath, long size) {
+ this.filePaths.add(filePath);
+ this.totalSize += size;
+ }
+
+ public long getTotalSize() {
+ return totalSize;
+ }
+
+ @Override
+ public String toString() {
+ return "Files: " + filePaths.size() + ", Total Size: " + totalSize;
+ }
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3ReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3ReaderFactory.java
new file mode 100644
index 0000000..1dd0c8b
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/aws/AwsS3ReaderFactory.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.record.reader.aws;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.asterix.external.input.record.reader.stream.StreamRecordReaderFactory;
+import org.apache.asterix.external.provider.StreamRecordReaderProvider;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.application.IServiceContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+
+public class AwsS3ReaderFactory extends StreamRecordReaderFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ private static final List<String> recordReaderNames =
+ Collections.singletonList(ExternalDataConstants.KEY_ADAPTER_NAME_AWS_S3);
+
+ @Override
+ public List<String> getRecordReaderNames() {
+ return recordReaderNames;
+ }
+
+ @Override
+ public DataSourceType getDataSourceType() {
+ return DataSourceType.RECORDS;
+ }
+
+ @Override
+ public Class<?> getRecordClass() {
+ return char[].class;
+ }
+
+ @Override
+ public AlgebricksAbsolutePartitionConstraint getPartitionConstraint() throws AlgebricksException {
+ return streamFactory.getPartitionConstraint();
+ }
+
+ @Override
+ public void configure(IServiceContext ctx, Map<String, String> configuration, IWarningCollector warningCollector)
+ throws AlgebricksException, HyracksDataException {
+ this.configuration = configuration;
+
+ // Stream factory
+ streamFactory = new AwsS3InputStreamFactory();
+ streamFactory.configure(ctx, configuration, warningCollector);
+
+ // record reader
+ recordReaderClazz = StreamRecordReaderProvider.getRecordReaderClazz(configuration);
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSLookupReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSLookupReaderFactory.java
index 98f78cc..8b258e4 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSLookupReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/hdfs/HDFSLookupReaderFactory.java
@@ -34,6 +34,7 @@
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
import org.apache.hyracks.hdfs.dataflow.ConfFactory;
public class HDFSLookupReaderFactory<T> implements ILookupReaderFactory<T> {
@@ -60,7 +61,8 @@
}
@Override
- public void configure(IServiceContext serviceCtx, Map<String, String> configuration) throws AsterixException {
+ public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
+ IWarningCollector warningCollector) throws AsterixException {
this.serviceCtx = serviceCtx;
this.configuration = configuration;
JobConf conf = HDFSUtils.configureHDFSJobConf(configuration);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/http/HttpServerRecordReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/http/HttpServerRecordReaderFactory.java
index d14071f..5954d74 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/http/HttpServerRecordReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/http/HttpServerRecordReaderFactory.java
@@ -33,6 +33,7 @@
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
import org.apache.hyracks.http.server.HttpServerConfigBuilder;
public class HttpServerRecordReaderFactory implements IRecordReaderFactory<char[]> {
@@ -86,7 +87,8 @@
}
@Override
- public void configure(IServiceContext ctx, Map<String, String> configuration) throws AlgebricksException {
+ public void configure(IServiceContext ctx, Map<String, String> configuration, IWarningCollector warningCollector)
+ throws AlgebricksException {
this.configurations = configuration;
// necessary configs
addrValue = getConfigurationValue(KEY_CONFIGURATION_ADDRESSES, true);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/rss/RSSRecordReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/rss/RSSRecordReaderFactory.java
index 2c07c35..b52636a 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/rss/RSSRecordReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/rss/RSSRecordReaderFactory.java
@@ -35,6 +35,7 @@
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
import com.rometools.rome.feed.synd.SyndEntry;
@@ -60,7 +61,8 @@
}
@Override
- public void configure(IServiceContext serviceContext, Map<String, String> configuration) {
+ public void configure(IServiceContext serviceContext, Map<String, String> configuration,
+ IWarningCollector warningCollector) {
this.serviceContext = serviceContext;
String url = configuration.get(ExternalDataConstants.KEY_RSS_URL);
if (url == null) {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/EmptyLineSeparatedRecordReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/EmptyLineSeparatedRecordReader.java
index 07b6250..b697b05 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/EmptyLineSeparatedRecordReader.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/EmptyLineSeparatedRecordReader.java
@@ -26,6 +26,7 @@
import org.apache.asterix.external.api.AsterixInputStream;
import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
public class EmptyLineSeparatedRecordReader extends StreamRecordReader {
@@ -135,8 +136,8 @@
}
@Override
- public void configure(AsterixInputStream inputStream, Map<String, String> config) {
- super.configure(inputStream);
+ public void configure(IHyracksTaskContext ctx, AsterixInputStream inputStream, Map<String, String> config) {
+ super.configure(inputStream, config);
this.config = config;
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/LineRecordReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/LineRecordReader.java
index 0b41d4b..4b86142 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/LineRecordReader.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/LineRecordReader.java
@@ -23,37 +23,49 @@
import java.util.Collections;
import java.util.List;
import java.util.Map;
+import java.util.function.LongSupplier;
import org.apache.asterix.external.api.AsterixInputStream;
import org.apache.asterix.external.util.ExternalDataConstants;
import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
public class LineRecordReader extends StreamRecordReader {
- private boolean hasHeader;
+ protected boolean hasHeader;
protected boolean prevCharCR;
protected int newlineLength;
- protected int recordNumber = 0;
- protected boolean nextIsHeader = false;
- private static final List<String> recordReaderFormats = Collections.unmodifiableList(
- Arrays.asList(ExternalDataConstants.FORMAT_DELIMITED_TEXT, ExternalDataConstants.FORMAT_CSV));
+ protected long beginLineNumber = 1;
+ protected long lineNumber = 1;
+ protected boolean newSource = false;
+ private static final List<String> recordReaderFormats =
+ Collections.unmodifiableList(Arrays.asList(ExternalDataConstants.FORMAT_DELIMITED_TEXT,
+ ExternalDataConstants.FORMAT_CSV, ExternalDataConstants.FORMAT_TSV));
private static final String REQUIRED_CONFIGS = "";
@Override
- public void configure(AsterixInputStream inputStream, Map<String, String> config) throws HyracksDataException {
- super.configure(inputStream);
+ public void configure(IHyracksTaskContext ctx, AsterixInputStream inputStream, Map<String, String> config)
+ throws HyracksDataException {
+ super.configure(inputStream, config);
this.hasHeader = ExternalDataUtils.hasHeader(config);
- if (hasHeader) {
- inputStream.setNotificationHandler(this);
- }
+ this.newSource = true;
+ inputStream.setNotificationHandler(this);
}
@Override
public void notifyNewSource() {
- if (hasHeader) {
- nextIsHeader = true;
- }
+ resetForNewSource();
+ }
+
+ @Override
+ public void resetForNewSource() {
+ super.resetForNewSource();
+ newSource = true;
+ beginLineNumber = 1;
+ lineNumber = 1;
+ prevCharCR = false;
+ newlineLength = 0;
}
@Override
@@ -89,6 +101,7 @@
* consuming it until we have a chance to look at the char that
* follows.
*/
+ beginLineNumber = lineNumber;
newlineLength = 0; //length of terminating newline
prevCharCR = false; //true of prev char was CR
record.reset();
@@ -99,42 +112,50 @@
startPosn = bufferPosn = 0;
bufferLength = reader.read(inputBuffer);
if (bufferLength <= 0) {
- if (readLength > 0) {
- record.endRecord();
- recordNumber++;
- return true;
+ if (readLength <= 0) {
+ close();
+ return false; //EOF
}
- close();
- return false; //EOF
+ record.endRecord();
+ break;
}
}
for (; bufferPosn < bufferLength; ++bufferPosn) { //search for newline
if (inputBuffer[bufferPosn] == ExternalDataConstants.LF) {
newlineLength = (prevCharCR) ? 2 : 1;
++bufferPosn; // at next invocation proceed from following byte
+ ++lineNumber;
break;
}
if (prevCharCR) { //CR + notLF, we are at notLF
+ ++lineNumber;
newlineLength = 1;
break;
}
prevCharCR = (inputBuffer[bufferPosn] == ExternalDataConstants.CR);
}
readLength = bufferPosn - startPosn;
- if (prevCharCR && newlineLength == 0) {
- --readLength; //CR at the end of the buffer
- prevCharCR = false;
- }
if (readLength > 0) {
record.append(inputBuffer, startPosn, readLength);
}
} while (newlineLength == 0);
- if (nextIsHeader) {
- nextIsHeader = false;
+ if (record.isEmptyRecord()) {
continue;
}
- recordNumber++;
+ if (newSource && hasHeader) {
+ newSource = false;
+ continue;
+ }
return true;
}
}
+
+ @Override
+ public LongSupplier getLineNumber() {
+ return this::getBeginLineNumber;
+ }
+
+ private long getBeginLineNumber() {
+ return beginLineNumber;
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/QuotedLineRecordReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/QuotedLineRecordReader.java
index 4c4128a..3a502d0 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/QuotedLineRecordReader.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/QuotedLineRecordReader.java
@@ -18,35 +18,60 @@
*/
package org.apache.asterix.external.input.record.reader.stream;
+import static org.apache.asterix.external.util.ExternalDataConstants.REC_ENDED_AT_EOF;
+
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
-import org.apache.asterix.common.exceptions.ExceptionUtils;
import org.apache.asterix.external.api.AsterixInputStream;
import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.ParseUtil;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
public class QuotedLineRecordReader extends LineRecordReader {
private char quote;
+ private char escape;
private boolean prevCharEscape;
+ private int readLength;
private boolean inQuote;
+ private IWarningCollector warnings;
private static final List<String> recordReaderFormats = Collections.unmodifiableList(
Arrays.asList(ExternalDataConstants.FORMAT_DELIMITED_TEXT, ExternalDataConstants.FORMAT_CSV));
- private static final String REQUIRED_CONFIGS = "quote";
+ private static final String REQUIRED_CONFIGS = ExternalDataConstants.KEY_QUOTE;
@Override
- public void configure(AsterixInputStream inputStream, Map<String, String> config) throws HyracksDataException {
- super.configure(inputStream, config);
+ public void configure(IHyracksTaskContext ctx, AsterixInputStream inputStream, Map<String, String> config)
+ throws HyracksDataException {
+ super.configure(ctx, inputStream, config);
+ this.warnings = ctx.getWarningCollector();
String quoteString = config.get(ExternalDataConstants.KEY_QUOTE);
- if (quoteString.length() != 1) {
- throw new HyracksDataException(ExceptionUtils.incorrectParameterMessage(ExternalDataConstants.KEY_QUOTE,
- ExternalDataConstants.PARAMETER_OF_SIZE_ONE, quoteString));
- }
+ ExternalDataUtils.validateChar(quoteString, ExternalDataConstants.KEY_QUOTE);
this.quote = quoteString.charAt(0);
+ this.escape = ExternalDataUtils.validateGetEscape(config);
+ }
+
+ @Override
+ public void notifyNewSource() {
+ if (!record.isEmptyRecord() && warnings.shouldWarn()) {
+ ParseUtil.warn(warnings, getPreviousStreamName(), lineNumber, 0, REC_ENDED_AT_EOF);
+ }
+ // restart for a new record from a new source
+ resetForNewSource();
+ }
+
+ @Override
+ public void resetForNewSource() {
+ super.resetForNewSource();
+ prevCharEscape = false;
+ readLength = 0;
+ inQuote = false;
}
@Override
@@ -65,11 +90,12 @@
if (done) {
return false;
}
+ beginLineNumber = lineNumber;
newlineLength = 0;
prevCharCR = false;
prevCharEscape = false;
record.reset();
- int readLength = 0;
+ readLength = 0;
inQuote = false;
do {
int startPosn = bufferPosn;
@@ -77,23 +103,33 @@
startPosn = bufferPosn = 0;
bufferLength = reader.read(inputBuffer);
if (bufferLength <= 0) {
- {
- if (readLength > 0) {
- if (inQuote) {
- throw new IOException("malformed input record ended inside quote");
- }
- record.endRecord();
- recordNumber++;
- return true;
+ // reached end of stream
+ if (readLength <= 0 || inQuote) {
+ // haven't read anything previously OR have read and in the middle and hit the end
+ if (inQuote && warnings.shouldWarn()) {
+ ParseUtil.warn(warnings, getDataSourceName().get(), lineNumber, 0, REC_ENDED_AT_EOF);
}
close();
return false;
}
+ record.endRecord();
+ break;
}
}
+ boolean maybeInQuote = false;
for (; bufferPosn < bufferLength; ++bufferPosn) {
+ char ch = inputBuffer[bufferPosn];
+ // count lines here since we need to also count the lines inside quotes
+ if (ch == ExternalDataConstants.LF || prevCharCR) {
+ lineNumber++;
+ }
+ if (ch == quote && escape == quote) {
+ inQuote |= maybeInQuote;
+ prevCharEscape |= maybeInQuote;
+ }
+ maybeInQuote = false;
if (!inQuote) {
- if (inputBuffer[bufferPosn] == ExternalDataConstants.LF) {
+ if (ch == ExternalDataConstants.LF) {
newlineLength = (prevCharCR) ? 2 : 1;
++bufferPosn;
break;
@@ -102,40 +138,33 @@
newlineLength = 1;
break;
}
- prevCharCR = (inputBuffer[bufferPosn] == ExternalDataConstants.CR);
- if (inputBuffer[bufferPosn] == quote) {
- if (!prevCharEscape) {
- inQuote = true;
- }
- }
- if (prevCharEscape) {
- prevCharEscape = false;
- } else {
- prevCharEscape = inputBuffer[bufferPosn] == ExternalDataConstants.ESCAPE;
- }
+ // if this is an opening quote, mark it
+ inQuote = ch == quote && !prevCharEscape;
+ // the escape != quote is for making an opening quote not an escape
+ prevCharEscape = ch == escape && !prevCharEscape && escape != quote;
} else {
- // only look for next quote
- if (inputBuffer[bufferPosn] == quote) {
- if (!prevCharEscape) {
- inQuote = false;
- }
+ // if quote == escape and current char is quote, then it could be closing or escaping
+ if (ch == quote && !prevCharEscape) {
+ // this is most likely a closing quote. the outcome depends on the next char
+ inQuote = false;
+ maybeInQuote = true;
}
- prevCharEscape = inputBuffer[bufferPosn] == ExternalDataConstants.ESCAPE;
+ prevCharEscape = ch == escape && !prevCharEscape && escape != quote;
}
+ prevCharCR = (ch == ExternalDataConstants.CR);
}
readLength = bufferPosn - startPosn;
- if (prevCharCR && newlineLength == 0) {
- --readLength;
- }
if (readLength > 0) {
record.append(inputBuffer, startPosn, readLength);
}
} while (newlineLength == 0);
- if (nextIsHeader) {
- nextIsHeader = false;
+ if (record.isEmptyRecord()) {
continue;
}
- recordNumber++;
+ if (newSource && hasHeader) {
+ newSource = false;
+ continue;
+ }
return true;
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/SemiStructuredRecordReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/SemiStructuredRecordReader.java
index 883f0ee..2ff5cfa 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/SemiStructuredRecordReader.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/SemiStructuredRecordReader.java
@@ -18,63 +18,93 @@
*/
package org.apache.asterix.external.input.record.reader.stream;
+import static org.apache.asterix.external.util.ExternalDataConstants.CLOSING_BRACKET;
+import static org.apache.asterix.external.util.ExternalDataConstants.COMMA;
+import static org.apache.asterix.external.util.ExternalDataConstants.CR;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_RECORD_END;
+import static org.apache.asterix.external.util.ExternalDataConstants.LF;
+import static org.apache.asterix.external.util.ExternalDataConstants.OPEN_BRACKET;
+import static org.apache.asterix.external.util.ExternalDataConstants.REC_ENDED_AT_EOF;
+import static org.apache.asterix.external.util.ExternalDataConstants.SPACE;
+import static org.apache.asterix.external.util.ExternalDataConstants.TAB;
+
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
+import java.util.function.LongSupplier;
import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.exceptions.ExceptionUtils;
import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.external.api.AsterixInputStream;
import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
+import org.apache.asterix.external.util.ParseUtil;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
public class SemiStructuredRecordReader extends StreamRecordReader {
+ private enum State {
+ TOP_LEVEL, // valid chars at this state: '{' or '[' to start a new record or array of records
+ ARRAY, // valid chars at this state: '{' or ']' to start the first nested record or close the array
+ NESTED_OBJECT, // valid chars at this state: ',' or ']' to close the array or expect another nested record
+ AFTER_COMMA // valid chars at this state: '{' to start a new nested record
+ }
+
+ private IWarningCollector warnings;
private int depth;
private boolean prevCharEscape;
private boolean inString;
private char recordStart;
private char recordEnd;
- private int recordNumber = 0;
+ private boolean hasStarted;
+ private boolean hasFinished;
+ private boolean isLastCharCR;
+ private State state = State.TOP_LEVEL;
+ private long beginLineNumber = 1;
+ private long lineNumber = 1;
+
private static final List<String> recordReaderFormats = Collections.unmodifiableList(
Arrays.asList(ExternalDataConstants.FORMAT_ADM, ExternalDataConstants.FORMAT_JSON_LOWER_CASE,
ExternalDataConstants.FORMAT_JSON_UPPER_CASE, ExternalDataConstants.FORMAT_SEMISTRUCTURED));
private static final String REQUIRED_CONFIGS = "";
@Override
- public void configure(AsterixInputStream stream, Map<String, String> config) throws HyracksDataException {
- super.configure(stream);
- String recStartString = config.get(ExternalDataConstants.KEY_RECORD_START);
- String recEndString = config.get(ExternalDataConstants.KEY_RECORD_END);
+ public void configure(IHyracksTaskContext ctx, AsterixInputStream stream, Map<String, String> config)
+ throws HyracksDataException {
+ super.configure(stream, config);
+ stream.setNotificationHandler(this);
+ warnings = ctx.getWarningCollector();
// set record opening char
- if (recStartString != null) {
- if (recStartString.length() != 1) {
- throw new HyracksDataException(
- ExceptionUtils.incorrectParameterMessage(ExternalDataConstants.KEY_RECORD_START,
- ExternalDataConstants.PARAMETER_OF_SIZE_ONE, recStartString));
- }
- recordStart = recStartString.charAt(0);
- } else {
- recordStart = ExternalDataConstants.DEFAULT_RECORD_START;
- }
+ recordStart = ExternalDataUtils.validateGetRecordStart(config);
// set record ending char
- if (recEndString != null) {
- if (recEndString.length() != 1) {
- throw new HyracksDataException(
- ExceptionUtils.incorrectParameterMessage(ExternalDataConstants.KEY_RECORD_END,
- ExternalDataConstants.PARAMETER_OF_SIZE_ONE, recEndString));
- }
- recordEnd = recEndString.charAt(0);
- } else {
- recordEnd = ExternalDataConstants.DEFAULT_RECORD_END;
+ recordEnd = ExternalDataUtils.validateGetRecordEnd(config);
+ if (recordStart == recordEnd) {
+ throw new RuntimeDataException(ErrorCode.INVALID_REQ_PARAM_VAL, KEY_RECORD_END, recordEnd);
}
}
- public int getRecordNumber() {
- return recordNumber;
+ @Override
+ public void notifyNewSource() {
+ if (hasStarted && warnings.shouldWarn()) {
+ ParseUtil.warn(warnings, getPreviousStreamName(), lineNumber, 0, REC_ENDED_AT_EOF);
+ }
+ beginLineNumber = 1;
+ lineNumber = 1;
+ state = State.TOP_LEVEL;
+ resetForNewRecord();
+ }
+
+ @Override
+ public LongSupplier getLineNumber() {
+ return this::getBeginLineNumber;
+ }
+
+ private long getBeginLineNumber() {
+ return beginLineNumber;
}
@Override
@@ -82,34 +112,52 @@
if (done) {
return false;
}
- record.reset();
- boolean hasStarted = false;
- boolean hasFinished = false;
- prevCharEscape = false;
- inString = false;
- depth = 0;
+ resetForNewRecord();
+ beginLineNumber = lineNumber;
do {
int startPosn = bufferPosn; // starting from where we left off the last time
if (bufferPosn >= bufferLength) {
startPosn = bufferPosn = 0;
bufferLength = reader.read(inputBuffer);
if (bufferLength < 0) {
+ if (hasStarted && warnings.shouldWarn()) {
+ ParseUtil.warn(warnings, getDataSourceName().get(), lineNumber, 0, REC_ENDED_AT_EOF);
+ }
close();
return false; // EOF
}
}
if (!hasStarted) {
for (; bufferPosn < bufferLength; ++bufferPosn) { // search for record begin
- if (inputBuffer[bufferPosn] == recordStart) {
+ char c = inputBuffer[bufferPosn];
+ if (c == LF || isLastCharCR) {
+ lineNumber++;
+ }
+ isLastCharCR = c == CR;
+ if (c == SPACE || c == TAB || c == LF || c == CR) {
+ continue;
+ }
+ if (c == recordStart && state != State.NESTED_OBJECT) {
+ // '{' is allowed at the top level, after '[' and after ','
+ if (state == State.ARRAY || state == State.AFTER_COMMA) {
+ state = State.NESTED_OBJECT;
+ }
+ beginLineNumber = lineNumber;
startPosn = bufferPosn;
hasStarted = true;
depth = 1;
- ++bufferPosn; // at next invocation proceed from following byte
+ ++bufferPosn;
break;
- } else if (inputBuffer[bufferPosn] != ExternalDataConstants.SPACE
- && inputBuffer[bufferPosn] != ExternalDataConstants.TAB
- && inputBuffer[bufferPosn] != ExternalDataConstants.LF
- && inputBuffer[bufferPosn] != ExternalDataConstants.CR) {
+ } else if (c == OPEN_BRACKET && state == State.TOP_LEVEL) {
+ // '[' is allowed at the top level only
+ state = State.ARRAY;
+ } else if (c == CLOSING_BRACKET && (state == State.ARRAY || state == State.NESTED_OBJECT)) {
+ // ']' is allowed after '[' and after capturing a record in an array
+ state = State.TOP_LEVEL;
+ } else if (c == COMMA && state == State.NESTED_OBJECT) {
+ // ',' is allowed after capturing a record in an array
+ state = State.AFTER_COMMA;
+ } else {
// corrupted file. clear the buffer and stop reading
reader.reset();
bufferPosn = bufferLength = 0;
@@ -118,23 +166,23 @@
}
}
if (hasStarted) {
- for (; bufferPosn < bufferLength; ++bufferPosn) { // search for record begin
+ for (; bufferPosn < bufferLength; ++bufferPosn) {
+ char c = inputBuffer[bufferPosn];
+ if (c == LF || isLastCharCR) {
+ lineNumber++;
+ }
if (inString) {
// we are in a string, we only care about the string end
- if (inputBuffer[bufferPosn] == ExternalDataConstants.QUOTE && !prevCharEscape) {
+ if (c == ExternalDataConstants.QUOTE && !prevCharEscape) {
inString = false;
}
- if (prevCharEscape) {
- prevCharEscape = false;
- } else {
- prevCharEscape = inputBuffer[bufferPosn] == ExternalDataConstants.ESCAPE;
- }
+ prevCharEscape = c == ExternalDataConstants.ESCAPE && !prevCharEscape;
} else {
- if (inputBuffer[bufferPosn] == ExternalDataConstants.QUOTE) {
+ if (c == ExternalDataConstants.QUOTE) {
inString = true;
- } else if (inputBuffer[bufferPosn] == recordStart) {
+ } else if (c == recordStart) {
depth += 1;
- } else if (inputBuffer[bufferPosn] == recordEnd) {
+ } else if (c == recordEnd) {
depth -= 1;
if (depth == 0) {
hasFinished = true;
@@ -143,22 +191,22 @@
}
}
}
+ isLastCharCR = c == CR;
}
- }
- int appendLength = bufferPosn - startPosn;
- if (appendLength > 0) {
- try {
- record.append(inputBuffer, startPosn, appendLength);
- } catch (IOException e) {
- reader.reset();
- bufferPosn = bufferLength = 0;
- throw new RuntimeDataException(ErrorCode.RECORD_READER_MALFORMED_INPUT_STREAM);
+ int appendLength = bufferPosn - startPosn;
+ if (appendLength > 0) {
+ try {
+ record.append(inputBuffer, startPosn, appendLength);
+ } catch (RuntimeDataException e) {
+ reader.reset();
+ bufferPosn = bufferLength = 0;
+ throw e;
+ }
}
}
} while (!hasFinished);
record.endRecord();
- recordNumber++;
return true;
}
@@ -172,14 +220,13 @@
return REQUIRED_CONFIGS;
}
- @Override
- public boolean stop() {
- try {
- reader.stop();
- } catch (Exception e) {
- e.printStackTrace();
- return false;
- }
- return true;
+ private void resetForNewRecord() {
+ record.reset();
+ hasStarted = false;
+ hasFinished = false;
+ prevCharEscape = false;
+ isLastCharCR = false;
+ inString = false;
+ depth = 0;
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/StreamRecordReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/StreamRecordReader.java
index 5629f48..1e003a1 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/StreamRecordReader.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/StreamRecordReader.java
@@ -18,9 +18,13 @@
*/
package org.apache.asterix.external.input.record.reader.stream;
+import static org.apache.asterix.external.util.ExternalDataConstants.EMPTY_STRING;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_REDACT_WARNINGS;
+
import java.io.IOException;
import java.util.List;
import java.util.Map;
+import java.util.function.Supplier;
import org.apache.asterix.external.api.AsterixInputStream;
import org.apache.asterix.external.api.IRawRecord;
@@ -29,8 +33,9 @@
import org.apache.asterix.external.dataflow.AbstractFeedDataFlowController;
import org.apache.asterix.external.input.record.CharArrayRecord;
import org.apache.asterix.external.input.stream.AsterixInputStreamReader;
-import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ExternalDataUtils;
import org.apache.asterix.external.util.FeedLogManager;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
public abstract class StreamRecordReader implements IRecordReader<char[]>, IStreamNotificationHandler {
@@ -41,11 +46,18 @@
protected int bufferPosn = 0;
protected boolean done = false;
protected FeedLogManager feedLogManager;
+ private Supplier<String> dataSourceName = EMPTY_STRING;
+ private Supplier<String> previousDataSourceName = EMPTY_STRING;
- public void configure(AsterixInputStream inputStream) {
- this.reader = new AsterixInputStreamReader(inputStream);
+ public void configure(AsterixInputStream inputStream, Map<String, String> config) {
+ int bufferSize = ExternalDataUtils.getOrDefaultBufferSize(config);
+ this.reader = new AsterixInputStreamReader(inputStream, bufferSize);
record = new CharArrayRecord();
- inputBuffer = new char[ExternalDataConstants.DEFAULT_BUFFER_SIZE];
+ inputBuffer = new char[bufferSize];
+ if (!ExternalDataUtils.isTrue(config, KEY_REDACT_WARNINGS)) {
+ this.dataSourceName = reader::getStreamName;
+ this.previousDataSourceName = reader::getPreviousStreamName;
+ }
}
@Override
@@ -55,10 +67,13 @@
@Override
public void close() throws IOException {
- if (!done) {
- reader.close();
+ try {
+ if (!done) {
+ reader.close();
+ }
+ } finally {
+ done = true;
}
- done = true;
}
@Override
@@ -96,10 +111,23 @@
throw new UnsupportedOperationException();
}
+ protected void resetForNewSource() {
+ record.reset();
+ }
+
+ @Override
+ public final Supplier<String> getDataSourceName() {
+ return dataSourceName;
+ }
+
+ String getPreviousStreamName() {
+ return previousDataSourceName.get();
+ }
+
public abstract List<String> getRecordReaderFormats();
public abstract String getRequiredConfigs();
- public abstract void configure(AsterixInputStream inputStream, Map<String, String> config)
+ public abstract void configure(IHyracksTaskContext ctx, AsterixInputStream inputStream, Map<String, String> config)
throws HyracksDataException;
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/StreamRecordReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/StreamRecordReaderFactory.java
index 076842e..c0a1b38 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/StreamRecordReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/stream/StreamRecordReaderFactory.java
@@ -39,6 +39,7 @@
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
public class StreamRecordReaderFactory implements IRecordReaderFactory<char[]> {
@@ -81,11 +82,11 @@
}
@Override
- public void configure(IServiceContext serviceCtx, Map<String, String> configuration)
- throws HyracksDataException, AlgebricksException {
+ public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
+ IWarningCollector warningCollector) throws HyracksDataException, AlgebricksException {
this.configuration = configuration;
configureInputStreamFactory(configuration);
- streamFactory.configure(serviceCtx, configuration);
+ streamFactory.configure(serviceCtx, configuration, warningCollector);
recordReaderClazz = StreamRecordReaderProvider.getRecordReaderClazz(configuration);
}
@@ -95,7 +96,7 @@
try {
StreamRecordReader streamRecordReader =
(StreamRecordReader) recordReaderClazz.getConstructor().newInstance();
- streamRecordReader.configure(streamFactory.createInputStream(ctx, partition), configuration);
+ streamRecordReader.configure(ctx, streamFactory.createInputStream(ctx, partition), configuration);
return streamRecordReader;
} catch (InstantiationException | IllegalAccessException | InvocationTargetException
| NoSuchMethodException e) {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/twitter/TwitterRecordReaderFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/twitter/TwitterRecordReaderFactory.java
index 43a3816..0fd0b95 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/twitter/TwitterRecordReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/record/reader/twitter/TwitterRecordReaderFactory.java
@@ -38,6 +38,7 @@
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -78,7 +79,8 @@
}
@Override
- public void configure(IServiceContext serviceCtx, Map<String, String> configuration) throws AsterixException {
+ public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
+ IWarningCollector warningCollector) throws AsterixException {
try {
Class.forName("twitter4j.Twitter");
} catch (ClassNotFoundException e) {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AbstractMultipleInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AbstractMultipleInputStream.java
new file mode 100644
index 0000000..8f032d8
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AbstractMultipleInputStream.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.stream;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.asterix.external.api.AsterixInputStream;
+import org.apache.asterix.external.api.IStreamNotificationHandler;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * Base class for a source stream that is composed of multiple separate input streams. Reading proceeds one stream at
+ * a time.
+ */
+public abstract class AbstractMultipleInputStream extends AsterixInputStream {
+
+ protected InputStream in;
+ private byte lastByte;
+
+ protected AbstractMultipleInputStream() {
+ }
+
+ /**
+ * Closes the current input stream and opens the next one, if any. Implementations should call
+ * {@link IStreamNotificationHandler#notifyNewSource()} using {@link #notificationHandler} if there exists a
+ * notification handler and the handler needs to know when a new input stream has started. Obviously, this method
+ * should populate the {@link #in} upon successfully opening the stream.
+ */
+ protected abstract boolean advance() throws IOException;
+
+ @Override
+ public int read() throws IOException {
+ throw new HyracksDataException(
+ "read() is not supported with this stream. use read(byte[] b, int off, int len)");
+ }
+
+ @Override
+ public final int read(byte[] b, int off, int len) throws IOException {
+ if (in == null) {
+ if (!advance()) {
+ return -1;
+ }
+ }
+ int result = in.read(b, off, len);
+ if (result < 0 && (lastByte != ExternalDataConstants.BYTE_LF) && (lastByte != ExternalDataConstants.BYTE_CR)) {
+ // return a new line at the end of every file <--Might create problems for some cases
+ // depending on the parser implementation-->
+ lastByte = ExternalDataConstants.BYTE_LF;
+ b[off] = ExternalDataConstants.BYTE_LF;
+ return 1;
+ }
+ while ((result < 0) && advance()) {
+ result = in.read(b, off, len);
+ }
+ if (result > 0) {
+ lastByte = b[(off + result) - 1];
+ }
+ return result;
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AsterixInputStreamReader.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AsterixInputStreamReader.java
index 8e166c0..ddbd350 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AsterixInputStreamReader.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/AsterixInputStreamReader.java
@@ -27,21 +27,23 @@
import org.apache.asterix.external.api.AsterixInputStream;
import org.apache.asterix.external.dataflow.AbstractFeedDataFlowController;
-import org.apache.asterix.external.util.ExternalDataConstants;
import org.apache.asterix.external.util.FeedLogManager;
import org.apache.hyracks.api.exceptions.HyracksDataException;
public class AsterixInputStreamReader extends Reader {
- private AsterixInputStream in;
- private byte[] bytes = new byte[ExternalDataConstants.DEFAULT_BUFFER_SIZE];
- private ByteBuffer byteBuffer = ByteBuffer.wrap(bytes);
- private CharBuffer charBuffer = CharBuffer.allocate(ExternalDataConstants.DEFAULT_BUFFER_SIZE);
- private CharsetDecoder decoder;
+ private final AsterixInputStream in;
+ private final byte[] bytes;
+ private final ByteBuffer byteBuffer;
+ private final CharBuffer charBuffer;
+ private final CharsetDecoder decoder;
private boolean done = false;
private boolean remaining = false;
- public AsterixInputStreamReader(AsterixInputStream in) {
+ public AsterixInputStreamReader(AsterixInputStream in, int bufferSize) {
this.in = in;
+ this.bytes = new byte[bufferSize];
+ this.byteBuffer = ByteBuffer.wrap(bytes);
+ this.charBuffer = CharBuffer.allocate(bufferSize);
this.decoder = StandardCharsets.UTF_8.newDecoder();
this.byteBuffer.flip();
}
@@ -68,7 +70,7 @@
}
@Override
- public int read(char cbuf[], int offset, int length) throws IOException {
+ public int read(char[] cbuf, int offset, int length) throws IOException {
if (done) {
return -1;
}
@@ -84,7 +86,9 @@
} else {
// need to read more data
System.arraycopy(bytes, byteBuffer.position(), bytes, 0, byteBuffer.remaining());
+ len = 0; // reset to read more bytes
byteBuffer.position(byteBuffer.remaining());
+ byteBuffer.limit(byteBuffer.capacity()); //set limit to capacity for the new bytes
while (len == 0) {
len = in.read(bytes, byteBuffer.position(), bytes.length - byteBuffer.position());
}
@@ -125,4 +129,12 @@
public void reset() throws IOException {
byteBuffer.limit(0);
}
+
+ public String getStreamName() {
+ return in.getStreamName();
+ }
+
+ public String getPreviousStreamName() {
+ return in.getPreviousStreamName();
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/LocalFSInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/LocalFSInputStream.java
index 2207bd7..f9eade5 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/LocalFSInputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/LocalFSInputStream.java
@@ -18,15 +18,16 @@
*/
package org.apache.asterix.external.input.stream;
+import static org.apache.asterix.common.exceptions.ErrorCode.INPUT_RECORD_READER_CHAR_ARRAY_RECORD_TOO_LARGE;
+import static org.apache.asterix.common.exceptions.ErrorCode.RECORD_READER_MALFORMED_INPUT_STREAM;
+import static org.apache.hyracks.api.exceptions.ErrorCode.PARSING_ERROR;
+
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
-import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.exceptions.ExceptionUtils;
-import org.apache.asterix.external.api.AsterixInputStream;
import org.apache.asterix.external.dataflow.AbstractFeedDataFlowController;
-import org.apache.asterix.external.util.ExternalDataConstants;
import org.apache.asterix.external.util.FeedLogManager;
import org.apache.asterix.external.util.FileSystemWatcher;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -34,13 +35,12 @@
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
-public class LocalFSInputStream extends AsterixInputStream {
+public class LocalFSInputStream extends AbstractMultipleInputStream {
private static final Logger LOGGER = LogManager.getLogger();
private final FileSystemWatcher watcher;
- private FileInputStream in;
- private byte lastByte;
private File currentFile;
+ private String lastFileName = "";
public LocalFSInputStream(FileSystemWatcher watcher) {
this.watcher = watcher;
@@ -92,10 +92,12 @@
}
}
- /**
- * Closes the current input stream and opens the next one, if any.
- */
- private boolean advance() throws IOException {
+ @Override
+ protected boolean advance() throws IOException {
+ String tmpLastFileName = "";
+ if (currentFile != null) {
+ tmpLastFileName = currentFile.getPath();
+ }
closeFile();
currentFile = watcher.poll();
if (currentFile == null) {
@@ -106,6 +108,7 @@
}
if (currentFile != null) {
in = new FileInputStream(currentFile);
+ lastFileName = tmpLastFileName;
if (notificationHandler != null) {
notificationHandler.notifyNewSource();
}
@@ -115,37 +118,6 @@
}
@Override
- public int read() throws IOException {
- throw new HyracksDataException(
- "read() is not supported with this stream. use read(byte[] b, int off, int len)");
- }
-
- @Override
- public int read(byte[] b, int off, int len) throws IOException {
- if (in == null) {
- if (!advance()) {
- return -1;
- }
- }
- int result = in.read(b, off, len);
- while ((result < 0) && advance()) {
- // return a new line at the end of every file <--Might create problems for some cases
- // depending on the parser implementation-->
- if ((lastByte != ExternalDataConstants.BYTE_LF) && (lastByte != ExternalDataConstants.BYTE_LF)) {
- lastByte = ExternalDataConstants.BYTE_LF;
- b[off] = ExternalDataConstants.BYTE_LF;
- return 1;
- }
- // recursive call
- result = in.read(b, off, len);
- }
- if (result > 0) {
- lastByte = b[(off + result) - 1];
- }
- return result;
- }
-
- @Override
public boolean stop() throws Exception {
closeFile();
watcher.close();
@@ -158,24 +130,46 @@
return false;
}
Throwable root = ExceptionUtils.getRootCause(th);
- if (root instanceof HyracksDataException
- && ((HyracksDataException) root).getErrorCode() == ErrorCode.RECORD_READER_MALFORMED_INPUT_STREAM) {
- if (currentFile != null) {
- try {
- logManager.logRecord(currentFile.getAbsolutePath(), "Corrupted input file");
- } catch (IOException e) {
- LOGGER.log(Level.WARN, "Filed to write to feed log file", e);
- }
- LOGGER.log(Level.WARN, "Corrupted input file: " + currentFile.getAbsolutePath());
+ if (root instanceof HyracksDataException) {
+ HyracksDataException r = (HyracksDataException) root;
+ boolean advance = false;
+ if (r.matchesAny(RECORD_READER_MALFORMED_INPUT_STREAM, PARSING_ERROR)) {
+ logCorruptedInput();
+ advance = true;
+ } else if (r.matches(INPUT_RECORD_READER_CHAR_ARRAY_RECORD_TOO_LARGE)) {
+ advance = true;
}
- try {
- advance();
- return true;
- } catch (Exception e) {
- LOGGER.log(Level.WARN, "An exception was thrown while trying to skip a file", e);
+ if (advance) {
+ try {
+ advance();
+ return true;
+ } catch (Exception e) {
+ LOGGER.log(Level.WARN, "An exception was thrown while trying to skip a file", e);
+ }
}
}
LOGGER.log(Level.WARN, "Failed to recover from failure", th);
return false;
}
+
+ @Override
+ public String getStreamName() {
+ return currentFile == null ? "" : currentFile.getPath();
+ }
+
+ @Override
+ public String getPreviousStreamName() {
+ return lastFileName;
+ }
+
+ private void logCorruptedInput() {
+ if (currentFile != null) {
+ try {
+ logManager.logRecord(currentFile.getAbsolutePath(), "Corrupted input file");
+ } catch (IOException e) {
+ LOGGER.log(Level.WARN, "Filed to write to feed log file", e);
+ }
+ LOGGER.log(Level.WARN, "Corrupted input file: " + currentFile.getAbsolutePath());
+ }
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketServerInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketServerInputStream.java
index fb9a4a6..0fe81cd 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketServerInputStream.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/SocketServerInputStream.java
@@ -137,6 +137,11 @@
@Override
public boolean handleException(Throwable th) {
+ if (th instanceof RuntimeException) {
+ //Log runtime exception
+ //TODO Should we continue anyway?
+ LOGGER.error("Encountered an unexpected error", th);
+ }
try {
return accept();
} catch (IOException e) {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/LocalFSInputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/LocalFSInputStreamFactory.java
index 6d91067..58ef2a4 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/LocalFSInputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/LocalFSInputStreamFactory.java
@@ -43,6 +43,7 @@
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
import org.apache.hyracks.api.io.UnmanagedFileSplit;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
@@ -90,7 +91,8 @@
}
@Override
- public void configure(IServiceContext serviceCtx, Map<String, String> configuration) throws AsterixException {
+ public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
+ IWarningCollector warningCollector) throws AsterixException {
this.configuration = configuration;
String[] splits = configuration.get(ExternalDataConstants.KEY_PATH).split(",");
if (inputFileSplits == null) {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/SocketClientInputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/SocketClientInputStreamFactory.java
index bfc2cb7..5d2b2a6 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/SocketClientInputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/SocketClientInputStreamFactory.java
@@ -39,6 +39,7 @@
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
public class SocketClientInputStreamFactory implements IInputStreamFactory {
@@ -55,7 +56,8 @@
}
@Override
- public void configure(IServiceContext serviceCtx, Map<String, String> configuration) throws AsterixException {
+ public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
+ IWarningCollector warningCollector) throws AsterixException {
try {
this.serviceCtx = serviceCtx;
this.sockets = new ArrayList<>();
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/SocketServerInputStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/SocketServerInputStreamFactory.java
index ac3ac42..1bd08d9 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/SocketServerInputStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/SocketServerInputStreamFactory.java
@@ -36,6 +36,7 @@
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
public class SocketServerInputStreamFactory implements IInputStreamFactory {
@@ -43,7 +44,8 @@
private List<Pair<String, Integer>> sockets;
@Override
- public void configure(IServiceContext serviceCtx, Map<String, String> configuration) throws CompilationException {
+ public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
+ IWarningCollector warningCollector) throws CompilationException {
try {
sockets = FeedUtils.extractHostsPorts(configuration.get(ExternalDataConstants.KEY_MODE), serviceCtx,
configuration.get(ExternalDataConstants.KEY_SOCKETS));
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/TwitterFirehoseStreamFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/TwitterFirehoseStreamFactory.java
index c7b8633..2b0bb55 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/TwitterFirehoseStreamFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/input/stream/factory/TwitterFirehoseStreamFactory.java
@@ -31,6 +31,7 @@
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
/**
* Factory class for creating @see{TwitterFirehoseFeedAdapter}. The adapter
@@ -84,7 +85,8 @@
}
@Override
- public void configure(IServiceContext serviceCtx, Map<String, String> configuration) {
+ public void configure(IServiceContext serviceCtx, Map<String, String> configuration,
+ IWarningCollector warningCollector) {
this.serviceCtx = serviceCtx;
this.configuration = configuration;
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexModificationOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexModificationOperatorDescriptor.java
index ea99993..477f0fe 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexModificationOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalFilesIndexModificationOperatorDescriptor.java
@@ -23,6 +23,7 @@
import java.util.Map;
import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.external.indexing.ExternalFile;
import org.apache.asterix.external.indexing.FileIndexTupleTranslator;
import org.apache.hyracks.api.comm.IFrameWriter;
@@ -89,7 +90,7 @@
case NO_OP:
break;
default:
- throw HyracksDataException.create(ErrorCode.UNKNOWN_EXTERNAL_FILE_PENDING_OP, sourceLoc,
+ throw RuntimeDataException.create(ErrorCode.UNKNOWN_EXTERNAL_FILE_PENDING_OP, sourceLoc,
file.getPendingOp());
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalScanOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalScanOperatorDescriptor.java
index 081d49e..1d7623d 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalScanOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalScanOperatorDescriptor.java
@@ -18,14 +18,15 @@
*/
package org.apache.asterix.external.operators;
-import org.apache.asterix.external.api.IAdapterFactory;
-import org.apache.asterix.external.api.IDataSourceAdapter;
+import org.apache.asterix.common.external.IDataSourceAdapter;
+import org.apache.asterix.external.api.ITypedAdapterFactory;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.api.job.profiling.IOperatorStats;
import org.apache.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
import org.apache.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
@@ -37,10 +38,10 @@
private static final long serialVersionUID = 1L;
- private IAdapterFactory adapterFactory;
+ private ITypedAdapterFactory adapterFactory;
public ExternalScanOperatorDescriptor(JobSpecification spec, RecordDescriptor rDesc,
- IAdapterFactory dataSourceAdapterFactory) {
+ ITypedAdapterFactory dataSourceAdapterFactory) {
super(spec, 0, 1);
outRecDescs[0] = rDesc;
this.adapterFactory = dataSourceAdapterFactory;
@@ -53,13 +54,21 @@
return new AbstractUnaryOutputSourceOperatorNodePushable() {
+ private IOperatorStats stats;
+
@Override
public void initialize() throws HyracksDataException {
- IDataSourceAdapter adapter = null;
+ IDataSourceAdapter adapter;
+ if (ctx.getStatsCollector() != null) {
+ stats = ctx.getStatsCollector().getOrAddOperatorStats(getDisplayName());
+ }
try {
writer.open();
adapter = adapterFactory.createAdapter(ctx, partition);
adapter.start(partition, writer);
+ if (stats != null) {
+ stats.getTupleCounter().update(adapter.getProcessedTuples());
+ }
} catch (Exception e) {
writer.fail();
throw HyracksDataException.create(e);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorDescriptor.java
index 7a0341a..7c709fb 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorDescriptor.java
@@ -25,7 +25,7 @@
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.common.library.ILibraryManager;
-import org.apache.asterix.external.api.IAdapterFactory;
+import org.apache.asterix.external.api.ITypedAdapterFactory;
import org.apache.asterix.external.feed.api.IFeed;
import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
import org.apache.asterix.om.types.ARecordType;
@@ -57,7 +57,7 @@
private final FeedPolicyAccessor policyAccessor;
private final ARecordType adapterOutputType;
/** The adaptor factory that is used to create an instance of the feed adaptor **/
- private IAdapterFactory adaptorFactory;
+ private ITypedAdapterFactory adaptorFactory;
/** The library that contains the adapter in use. **/
private String adaptorLibraryName;
/**
@@ -68,7 +68,7 @@
/** The configuration parameters associated with the adapter. **/
private Map<String, String> adaptorConfiguration;
- public FeedIntakeOperatorDescriptor(JobSpecification spec, IFeed primaryFeed, IAdapterFactory adapterFactory,
+ public FeedIntakeOperatorDescriptor(JobSpecification spec, IFeed primaryFeed, ITypedAdapterFactory adapterFactory,
ARecordType adapterOutputType, FeedPolicyAccessor policyAccessor, RecordDescriptor rDesc) {
super(spec, 0, 1);
this.feedId = new EntityId(FEED_EXTENSION_NAME, primaryFeed.getDataverseName(), primaryFeed.getFeedName());
@@ -100,17 +100,18 @@
return new FeedIntakeOperatorNodePushable(ctx, feedId, adaptorFactory, partition, recordDescProvider, this);
}
- private IAdapterFactory createExternalAdapterFactory(IHyracksTaskContext ctx) throws HyracksDataException {
- IAdapterFactory adapterFactory;
+ private ITypedAdapterFactory createExternalAdapterFactory(IHyracksTaskContext ctx) throws HyracksDataException {
+ ITypedAdapterFactory adapterFactory;
INcApplicationContext runtimeCtx =
(INcApplicationContext) ctx.getJobletContext().getServiceContext().getApplicationContext();
ILibraryManager libraryManager = runtimeCtx.getLibraryManager();
ClassLoader classLoader = libraryManager.getLibraryClassLoader(feedId.getDataverse(), adaptorLibraryName);
if (classLoader != null) {
try {
- adapterFactory = (IAdapterFactory) (classLoader.loadClass(adaptorFactoryClassName).newInstance());
+ adapterFactory = (ITypedAdapterFactory) (classLoader.loadClass(adaptorFactoryClassName).newInstance());
adapterFactory.setOutputType(adapterOutputType);
- adapterFactory.configure(ctx.getJobletContext().getServiceContext(), adaptorConfiguration);
+ adapterFactory.configure(ctx.getJobletContext().getServiceContext(), adaptorConfiguration,
+ ctx.getWarningCollector());
} catch (Exception e) {
throw HyracksDataException.create(e);
}
@@ -128,11 +129,11 @@
return feedId;
}
- public IAdapterFactory getAdaptorFactory() {
+ public ITypedAdapterFactory getAdaptorFactory() {
return this.adaptorFactory;
}
- public void setAdaptorFactory(IAdapterFactory factory) {
+ public void setAdaptorFactory(ITypedAdapterFactory factory) {
this.adaptorFactory = factory;
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorNodePushable.java
index 98f75df..e1ba399 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/FeedIntakeOperatorNodePushable.java
@@ -23,7 +23,7 @@
import org.apache.asterix.active.ActiveRuntimeId;
import org.apache.asterix.active.ActiveSourceOperatorNodePushable;
import org.apache.asterix.active.EntityId;
-import org.apache.asterix.external.api.IAdapterFactory;
+import org.apache.asterix.external.api.ITypedAdapterFactory;
import org.apache.asterix.external.dataset.adapter.FeedAdapter;
import org.apache.hyracks.api.comm.IFrame;
import org.apache.hyracks.api.comm.VSizeFrame;
@@ -50,7 +50,7 @@
private final FeedAdapter adapter;
private boolean poisoned = false;
- public FeedIntakeOperatorNodePushable(IHyracksTaskContext ctx, EntityId feedId, IAdapterFactory adapterFactory,
+ public FeedIntakeOperatorNodePushable(IHyracksTaskContext ctx, EntityId feedId, ITypedAdapterFactory adapterFactory,
int partition, IRecordDescriptorProvider recordDescProvider,
FeedIntakeOperatorDescriptor feedIntakeOperatorDescriptor) throws HyracksDataException {
super(ctx, new ActiveRuntimeId(feedId, FeedIntakeOperatorNodePushable.class.getSimpleName(), partition));
@@ -135,7 +135,7 @@
taskThread.interrupt();
}
} catch (HyracksDataException hde) {
- if (hde.getComponent() == ErrorCode.HYRACKS && hde.getErrorCode() == ErrorCode.TIMEOUT) {
+ if (hde.matches(ErrorCode.TIMEOUT)) {
LOGGER.log(Level.WARN, runtimeId + " stop adapter timed out. interrupting the thread...", hde);
taskThread.interrupt();
} else {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ADMDataParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ADMDataParser.java
index 42caf12..4a46717 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ADMDataParser.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ADMDataParser.java
@@ -74,8 +74,8 @@
private final TokenImage tmpTokenImage = new TokenImage();
- private final String mismatchErrorMessage = "Mismatch Type, expecting a value of type ";
- private final String mismatchErrorMessage2 = " got a value of type ";
+ private static final String mismatchErrorMessage = "Mismatch Type, expecting a value of type ";
+ private static final String mismatchErrorMessage2 = " got a value of type ";
public ADMDataParser(ARecordType recordType, boolean isStream) {
this(null, recordType, isStream);
@@ -103,11 +103,11 @@
}
@Override
- public void parse(IRawRecord<? extends char[]> record, DataOutput out) throws HyracksDataException {
+ public boolean parse(IRawRecord<? extends char[]> record, DataOutput out) throws HyracksDataException {
try {
resetPools();
admLexer.setBuffer(record.get());
- parseAdmInstance(recordType, out);
+ return parseAdmInstance(recordType, out);
} catch (ParseException e) {
e.setLocation(filename, admLexer.getLine(), admLexer.getColumn());
throw e;
@@ -121,7 +121,7 @@
admLexer = new AdmLexer(new java.io.InputStreamReader(in));
}
- protected boolean parseAdmInstance(IAType objectType, DataOutput out) throws IOException {
+ private boolean parseAdmInstance(IAType objectType, DataOutput out) throws IOException {
int token = admLexer.next();
if (token == AdmLexer.TOKEN_EOF) {
return false;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/DelimitedDataParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/DelimitedDataParser.java
index 4e371c8..60e6e77 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/DelimitedDataParser.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/DelimitedDataParser.java
@@ -18,10 +18,16 @@
*/
package org.apache.asterix.external.parser;
+import static org.apache.asterix.external.util.ExternalDataConstants.EMPTY_FIELD;
+import static org.apache.asterix.external.util.ExternalDataConstants.INVALID_VAL;
+import static org.apache.asterix.external.util.ExternalDataConstants.MISSING_FIELDS;
+
import java.io.DataOutput;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
+import java.util.function.LongSupplier;
+import java.util.function.Supplier;
import org.apache.asterix.builders.IARecordBuilder;
import org.apache.asterix.builders.RecordBuilder;
@@ -31,11 +37,17 @@
import org.apache.asterix.external.api.IRawRecord;
import org.apache.asterix.external.api.IRecordDataParser;
import org.apache.asterix.external.api.IStreamDataParser;
+import org.apache.asterix.external.util.ExternalDataConstants;
+import org.apache.asterix.external.util.ParseUtil;
import org.apache.asterix.om.base.AMutableString;
+import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.utils.NonTaggedFormatUtil;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
import org.apache.hyracks.dataflow.common.data.parsers.IValueParser;
import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory;
@@ -43,23 +55,30 @@
public class DelimitedDataParser extends AbstractDataParser implements IStreamDataParser, IRecordDataParser<char[]> {
+ private final IWarningCollector warnings;
private final char fieldDelimiter;
private final char quote;
private final boolean hasHeader;
- private ARecordType recordType;
- private IARecordBuilder recBuilder;
- private ArrayBackedValueStorage fieldValueBuffer;
- private DataOutput fieldValueBufferOutput;
- private IValueParser[] valueParsers;
+ private final ARecordType recordType;
+ private final IARecordBuilder recBuilder;
+ private final ArrayBackedValueStorage fieldValueBuffer;
+ private final DataOutput fieldValueBufferOutput;
+ private final IValueParser[] valueParsers;
private FieldCursorForDelimitedDataParser cursor;
- private byte[] fieldTypeTags;
- private int[] fldIds;
- private ArrayBackedValueStorage[] nameBuffers;
- private boolean areAllNullFields;
+ private Supplier<String> dataSourceName;
+ private LongSupplier lineNumber;
+ private final byte[] fieldTypeTags;
+ private final int[] fldIds;
+ private final ArrayBackedValueStorage[] nameBuffers;
+ private final char[] nullChars;
- public DelimitedDataParser(IValueParserFactory[] valueParserFactories, char fieldDelimter, char quote,
- boolean hasHeader, ARecordType recordType, boolean isStreamParser) throws HyracksDataException {
- this.fieldDelimiter = fieldDelimter;
+ public DelimitedDataParser(IHyracksTaskContext ctx, IValueParserFactory[] valueParserFactories, char fieldDelimiter,
+ char quote, boolean hasHeader, ARecordType recordType, boolean isStreamParser, String nullString)
+ throws HyracksDataException {
+ this.dataSourceName = ExternalDataConstants.EMPTY_STRING;
+ this.lineNumber = ExternalDataConstants.NO_LINES;
+ this.warnings = ctx.getWarningCollector();
+ this.fieldDelimiter = fieldDelimiter;
this.quote = quote;
this.hasHeader = hasHeader;
this.recordType = recordType;
@@ -98,18 +117,22 @@
}
}
if (!isStreamParser) {
- cursor = new FieldCursorForDelimitedDataParser(null, fieldDelimiter, quote);
+ cursor = new FieldCursorForDelimitedDataParser(null, this.fieldDelimiter, quote, warnings,
+ this::getDataSourceName);
}
+ this.nullChars = nullString != null ? nullString.toCharArray() : null;
}
@Override
public boolean parse(DataOutput out) throws HyracksDataException {
try {
- while (cursor.nextRecord()) {
- parseRecord();
- if (!areAllNullFields) {
+ if (cursor.nextRecord()) {
+ if (parseRecord()) {
recBuilder.write(out, true);
return true;
+ } else {
+ // keeping the behaviour of throwing exception for stream parsers
+ throw new RuntimeDataException(ErrorCode.FAILED_TO_PARSE_RECORD);
}
}
return false;
@@ -118,43 +141,53 @@
}
}
- private void parseRecord() throws HyracksDataException {
+ private boolean parseRecord() throws HyracksDataException {
recBuilder.reset(recordType);
recBuilder.init();
- areAllNullFields = true;
for (int i = 0; i < valueParsers.length; ++i) {
try {
- if (!cursor.nextField()) {
- break;
+ FieldCursorForDelimitedDataParser.Result result = cursor.nextField();
+ switch (result) {
+ case OK:
+ break;
+ case END:
+ if (warnings.shouldWarn()) {
+ ParseUtil.warn(warnings, dataSourceName.get(), cursor.getLineCount(),
+ cursor.getFieldCount(), MISSING_FIELDS);
+ }
+ return false;
+ case ERROR:
+ return false;
+ default:
+ throw new IllegalStateException();
}
- } catch (IOException e) {
- throw HyracksDataException.create(e);
- }
- fieldValueBuffer.reset();
+ fieldValueBuffer.reset();
- try {
- if (cursor.fStart == cursor.fEnd && recordType.getFieldTypes()[i].getTypeTag() != ATypeTag.STRING
- && recordType.getFieldTypes()[i].getTypeTag() != ATypeTag.NULL) {
- // if the field is empty and the type is optional, insert
- // NULL. Note that string type can also process empty field as an
- // empty string
- if (!NonTaggedFormatUtil.isOptional(recordType.getFieldTypes()[i])) {
- throw new RuntimeDataException(ErrorCode.PARSER_DELIMITED_NONOPTIONAL_NULL, cursor.recordCount,
- cursor.fieldCount);
- }
+ if (nullChars != null && NonTaggedFormatUtil.isOptional(recordType.getFieldTypes()[i]) && fieldNull()) {
fieldValueBufferOutput.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
} else {
- fieldValueBufferOutput.writeByte(fieldTypeTags[i]);
- // Eliminate doule quotes in the field that we are going to parse
- if (cursor.isDoubleQuoteIncludedInThisField) {
- cursor.eliminateDoubleQuote(cursor.buffer, cursor.fStart, cursor.fEnd - cursor.fStart);
- cursor.fEnd -= cursor.doubleQuoteCount;
- cursor.isDoubleQuoteIncludedInThisField = false;
+ if (cursor.isFieldEmpty() && !canProcessEmptyField(recordType.getFieldTypes()[i])) {
+ if (warnings.shouldWarn()) {
+ ParseUtil.warn(warnings, dataSourceName.get(), cursor.getLineCount(),
+ cursor.getFieldCount(), EMPTY_FIELD);
+ }
+ return false;
}
- valueParsers[i].parse(cursor.buffer, cursor.fStart, cursor.fEnd - cursor.fStart,
- fieldValueBufferOutput);
- areAllNullFields = false;
+ fieldValueBufferOutput.writeByte(fieldTypeTags[i]);
+ // Eliminate double quotes in the field that we are going to parse
+ if (cursor.fieldHasDoubleQuote()) {
+ cursor.eliminateDoubleQuote();
+ }
+ boolean success = valueParsers[i].parse(cursor.getBuffer(), cursor.getFieldStart(),
+ cursor.getFieldLength(), fieldValueBufferOutput);
+ if (!success) {
+ if (warnings.shouldWarn()) {
+ ParseUtil.warn(warnings, dataSourceName.get(), cursor.getLineCount(),
+ cursor.getFieldCount(), INVALID_VAL);
+ }
+ return false;
+ }
}
if (fldIds[i] < 0) {
recBuilder.addField(nameBuffers[i], fieldValueBuffer);
@@ -165,35 +198,81 @@
throw HyracksDataException.create(e);
}
}
- }
-
- @Override
- public void parse(IRawRecord<? extends char[]> record, DataOutput out) throws HyracksDataException {
try {
- cursor.nextRecord(record.get(), record.size());
+ while (cursor.nextField() == FieldCursorForDelimitedDataParser.Result.OK) {
+ // keep reading and discarding the extra fields
+ }
+ return true;
} catch (IOException e) {
throw HyracksDataException.create(e);
}
- parseRecord();
- if (!areAllNullFields) {
+ }
+
+ @Override
+ public boolean parse(IRawRecord<? extends char[]> record, DataOutput out) throws HyracksDataException {
+ cursor.nextRecord(record.get(), record.size(), lineNumber.getAsLong());
+ if (parseRecord()) {
recBuilder.write(out, true);
+ return true;
}
+ return false;
}
@Override
public void setInputStream(InputStream in) throws IOException {
- cursor = new FieldCursorForDelimitedDataParser(new InputStreamReader(in), fieldDelimiter, quote);
- if (in != null && hasHeader) {
+ // TODO(ali): revisit this in regards to stream
+ cursor = new FieldCursorForDelimitedDataParser(new InputStreamReader(in), fieldDelimiter, quote, warnings,
+ this::getDataSourceName);
+ if (hasHeader) {
cursor.nextRecord();
- while (cursor.nextField()) {
- ;
+ FieldCursorForDelimitedDataParser.Result result;
+ do {
+ result = cursor.nextField();
+ } while (result == FieldCursorForDelimitedDataParser.Result.OK);
+ if (result == FieldCursorForDelimitedDataParser.Result.ERROR) {
+ throw new RuntimeDataException(ErrorCode.FAILED_TO_PARSE_RECORD);
}
}
}
@Override
public boolean reset(InputStream in) throws IOException {
- cursor = new FieldCursorForDelimitedDataParser(new InputStreamReader(in), fieldDelimiter, quote);
+ // TODO(ali): revisit this in regards to stream
+ cursor = new FieldCursorForDelimitedDataParser(new InputStreamReader(in), fieldDelimiter, quote, warnings,
+ this::getDataSourceName);
+ return true;
+ }
+
+ @Override
+ public void configure(Supplier<String> dataSourceName, LongSupplier lineNumber) {
+ this.dataSourceName = dataSourceName == null ? ExternalDataConstants.EMPTY_STRING : dataSourceName;
+ this.lineNumber = lineNumber == null ? ExternalDataConstants.NO_LINES : lineNumber;
+
+ }
+
+ private String getDataSourceName() {
+ return dataSourceName.get();
+ }
+
+ private static boolean canProcessEmptyField(IAType fieldType) {
+ IAType type = TypeComputeUtils.getActualType(fieldType);
+ // TODO(ali): investigate what it means for a field to have type NULL. there is no parser implemented for it
+ return type.getTypeTag() == ATypeTag.STRING || type.getTypeTag() == ATypeTag.NULL;
+ }
+
+ private boolean fieldNull() {
+ int fieldLength = cursor.getFieldLength();
+ int nullStringLength = nullChars.length;
+ if (fieldLength != nullStringLength) {
+ return false;
+ }
+ char[] fieldChars = cursor.getBuffer();
+ int fieldStart = cursor.getFieldStart();
+ for (int i = 0; i < fieldLength; i++) {
+ if (fieldChars[fieldStart + i] != nullChars[i]) {
+ return false;
+ }
+ }
return true;
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/HiveRecordParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/HiveRecordParser.java
index 1a88d08..9ab6c7d 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/HiveRecordParser.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/HiveRecordParser.java
@@ -110,7 +110,7 @@
}
@Override
- public void parse(IRawRecord<? extends Writable> record, DataOutput out) throws HyracksDataException {
+ public boolean parse(IRawRecord<? extends Writable> record, DataOutput out) throws HyracksDataException {
try {
Writable hiveRawRecord = record.get();
Object hiveObject = hiveSerde.deserialize(hiveRawRecord);
@@ -129,6 +129,7 @@
recBuilder.addField(i, fieldValueBuffer);
}
recBuilder.write(out, true);
+ return true;
} catch (Exception e) {
throw HyracksDataException.create(e);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/JSONDataParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/JSONDataParser.java
index 72f86c1..b2036c0 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/JSONDataParser.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/JSONDataParser.java
@@ -22,6 +22,8 @@
import java.io.IOException;
import java.io.InputStream;
import java.util.BitSet;
+import java.util.function.LongSupplier;
+import java.util.function.Supplier;
import org.apache.asterix.builders.IARecordBuilder;
import org.apache.asterix.builders.IAsterixListBuilder;
@@ -33,6 +35,7 @@
import org.apache.asterix.external.parser.jackson.ADMToken;
import org.apache.asterix.external.parser.jackson.GeometryCoParser;
import org.apache.asterix.external.parser.jackson.ParserContext;
+import org.apache.asterix.external.util.ExternalDataConstants;
import org.apache.asterix.om.base.ABoolean;
import org.apache.asterix.om.base.ANull;
import org.apache.asterix.om.base.AUnorderedList;
@@ -45,10 +48,13 @@
import org.apache.asterix.om.utils.RecordUtil;
import org.apache.asterix.runtime.exceptions.UnsupportedTypeException;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.util.ExceptionUtils;
import org.apache.hyracks.data.std.api.IMutableValueStorage;
import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonParseException;
import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonStreamContext;
import com.fasterxml.jackson.core.JsonToken;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.TreeTraversingParser;
@@ -63,6 +69,8 @@
protected final JsonFactory jsonFactory;
protected final ARecordType rootType;
protected final GeometryCoParser geometryCoParser;
+ private Supplier<String> dataSourceName;
+ private LongSupplier lineNumber;
protected JsonParser jsonParser;
@@ -81,6 +89,8 @@
//GeometyCoParser to parse GeoJSON objects to AsterixDB internal spatial types.
geometryCoParser = new GeometryCoParser(jsonParser);
parserContext = new ParserContext();
+ this.dataSourceName = ExternalDataConstants.EMPTY_STRING;
+ this.lineNumber = ExternalDataConstants.NO_LINES;
}
/*
@@ -90,15 +100,22 @@
*/
@Override
- public final void parse(IRawRecord<? extends char[]> record, DataOutput out) throws HyracksDataException {
+ public void configure(Supplier<String> dataSourceName, LongSupplier lineNumber) {
+ this.dataSourceName = dataSourceName == null ? ExternalDataConstants.EMPTY_STRING : dataSourceName;
+ this.lineNumber = lineNumber == null ? ExternalDataConstants.NO_LINES : lineNumber;
+ }
+
+ @Override
+ public final boolean parse(IRawRecord<? extends char[]> record, DataOutput out) throws HyracksDataException {
try {
//TODO(wyk): find a way to reset byte[] instead of creating a new parser for each record.
jsonParser = jsonFactory.createParser(record.get(), 0, record.size());
geometryCoParser.reset(jsonParser);
nextToken();
parseObject(rootType, out);
+ return true;
} catch (IOException e) {
- throw new RuntimeDataException(ErrorCode.RECORD_READER_MALFORMED_INPUT_STREAM, e);
+ throw createException(e);
}
}
@@ -459,4 +476,33 @@
}
}
+
+ private HyracksDataException createException(IOException e) {
+ if (jsonParser != null) {
+ String msg;
+ if (e instanceof JsonParseException) {
+ msg = ((JsonParseException) e).getOriginalMessage();
+ } else {
+ msg = ExceptionUtils.getRootCause(e).getMessage();
+ }
+ if (msg == null) {
+ msg = ErrorCode.RECORD_READER_MALFORMED_INPUT_STREAM.errorMessage();
+ }
+ long lineNum = lineNumber.getAsLong() + jsonParser.getCurrentLocation().getLineNr() - 1;
+ JsonStreamContext parsingContext = jsonParser.getParsingContext();
+ String fieldName = "N/A";
+ while (parsingContext != null) {
+ String currentFieldName = parsingContext.getCurrentName();
+ if (currentFieldName != null) {
+ fieldName = currentFieldName;
+ break;
+ }
+ parsingContext = parsingContext.getParent();
+ }
+
+ return HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.PARSING_ERROR,
+ dataSourceName.get(), lineNum, fieldName, msg);
+ }
+ return new RuntimeDataException(ErrorCode.RECORD_READER_MALFORMED_INPUT_STREAM, e);
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ParseException.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ParseException.java
index 5ccb7a3..e9f93c9 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ParseException.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/ParseException.java
@@ -22,6 +22,7 @@
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IError;
public class ParseException extends HyracksDataException {
private static final long serialVersionUID = 1L;
@@ -33,23 +34,31 @@
super(message);
}
- public ParseException(int errorCode, Serializable... param) {
- super(ErrorCode.ASTERIX, errorCode, ErrorCode.getErrorMessage(errorCode), param);
+ public ParseException(ErrorCode error, Throwable e, Serializable... param) {
+ super(error, e, null, param);
}
- public ParseException(int errorCode, Throwable e, Serializable... param) {
- super(ErrorCode.ASTERIX, errorCode, e, ErrorCode.getErrorMessage(errorCode), param);
- addSuppressed(e);
+ public ParseException(ErrorCode error, Serializable... param) {
+ this(error, null, param);
}
+ /**
+ * @deprecated use {@link IError} ctors when possible
+ */
public ParseException(Throwable cause) {
super(cause);
}
+ /**
+ * @deprecated use {@link IError} ctors when possible
+ */
public ParseException(String message, Throwable cause) {
super(message, cause);
}
+ /**
+ * @deprecated use {@link IError} ctors when possible
+ */
public ParseException(Throwable cause, String filename, int line, int column) {
super(cause);
setLocation(filename, line, column);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/RSSParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/RSSParser.java
index e260083..84a2d90 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/RSSParser.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/RSSParser.java
@@ -48,7 +48,7 @@
}
@Override
- public void parse(IRawRecord<? extends SyndEntry> record, DataOutput out) throws HyracksDataException {
+ public boolean parse(IRawRecord<? extends SyndEntry> record, DataOutput out) throws HyracksDataException {
SyndEntry entry = record.get();
tupleFieldValues[0] = String.valueOf(id);
tupleFieldValues[1] = entry.getTitle();
@@ -62,5 +62,6 @@
recordBuilder.init();
IDataParser.writeRecord(mutableRecord, out, recordBuilder);
id++;
+ return true;
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/RecordWithMetadataParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/RecordWithMetadataParser.java
index 6c9298e..820775c 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/RecordWithMetadataParser.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/RecordWithMetadataParser.java
@@ -20,6 +20,8 @@
import java.io.DataOutput;
import java.io.IOException;
+import java.util.function.LongSupplier;
+import java.util.function.Supplier;
import org.apache.asterix.builders.RecordBuilder;
import org.apache.asterix.external.api.IDataParser;
@@ -72,14 +74,15 @@
}
@Override
- public void parse(IRawRecord<? extends T> record, DataOutput out) throws HyracksDataException {
+ public boolean parse(IRawRecord<? extends T> record, DataOutput out) throws HyracksDataException {
try {
rwm = converter.convert(record);
if (rwm.getRecord().size() == 0) {
// null record
out.writeByte(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
+ return true;
} else {
- recordParser.parse(rwm.getRecord(), out);
+ return recordParser.parse(rwm.getRecord(), out);
}
} catch (IOException e) {
throw HyracksDataException.create(e);
@@ -110,4 +113,10 @@
public void appendLastParsedPrimaryKeyToTuple(ArrayTupleBuilder tb) throws HyracksDataException {
rwm.appendPrimaryKeyToTuple(tb);
}
+
+ @Override
+ public void configure(Supplier<String> dataSourceName, LongSupplier lineNumber) {
+ this.recordParser.configure(dataSourceName, lineNumber);
+ this.converter.configure(lineNumber);
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/RecordWithPKDataParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/RecordWithPKDataParser.java
index aa0db53..8c00a9f 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/RecordWithPKDataParser.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/RecordWithPKDataParser.java
@@ -37,15 +37,16 @@
}
@Override
- public void parse(IRawRecord<? extends T> record, DataOutput out) throws HyracksDataException {
+ public boolean parse(IRawRecord<? extends T> record, DataOutput out) throws HyracksDataException {
if (record.size() == 0) {
try {
out.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+ return true;
} catch (IOException e) {
throw HyracksDataException.create(e);
}
} else {
- recordParser.parse(record, out);
+ return recordParser.parse(record, out);
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/TweetParser.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/TweetParser.java
index 1cbf0be..4726a50 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/TweetParser.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/TweetParser.java
@@ -259,12 +259,13 @@
}
@Override
- public void parse(IRawRecord<? extends char[]> record, DataOutput out) throws HyracksDataException {
+ public boolean parse(IRawRecord<? extends char[]> record, DataOutput out) throws HyracksDataException {
try {
//TODO get rid of this temporary json
resetPools();
ObjectMapper om = new ObjectMapper();
writeRecord(om.readTree(record.getBytes()), out, recordType);
+ return true;
} catch (IOException e) {
throw HyracksDataException.create(e);
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/DelimitedDataParserFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/DelimitedDataParserFactory.java
index f406729..09f9697 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/DelimitedDataParserFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/DelimitedDataParserFactory.java
@@ -21,10 +21,7 @@
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
-import java.util.Map;
-import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.external.api.IExternalDataSourceFactory.DataSourceType;
import org.apache.asterix.external.api.IRecordDataParser;
import org.apache.asterix.external.api.IStreamDataParser;
@@ -40,20 +37,22 @@
private static final long serialVersionUID = 1L;
private static final List<String> parserFormats =
- Collections.unmodifiableList(Arrays.asList("csv", "delimited-text"));
+ Collections.unmodifiableList(Arrays.asList(ExternalDataConstants.FORMAT_CSV,
+ ExternalDataConstants.FORMAT_DELIMITED_TEXT, ExternalDataConstants.FORMAT_TSV));
@Override
public IRecordDataParser<char[]> createRecordParser(IHyracksTaskContext ctx) throws HyracksDataException {
- return createParser();
+ return createParser(ctx);
}
- private DelimitedDataParser createParser() throws HyracksDataException {
+ private DelimitedDataParser createParser(IHyracksTaskContext ctx) throws HyracksDataException {
IValueParserFactory[] valueParserFactories = ExternalDataUtils.getValueParserFactories(recordType);
- Character delimiter = DelimitedDataParserFactory.getDelimiter(configuration);
- char quote = DelimitedDataParserFactory.getQuote(configuration, delimiter);
+ char delimiter = ExternalDataUtils.validateGetDelimiter(configuration);
+ char quote = ExternalDataUtils.validateGetQuote(configuration, delimiter);
boolean hasHeader = ExternalDataUtils.hasHeader(configuration);
- return new DelimitedDataParser(valueParserFactories, delimiter, quote, hasHeader, recordType,
- ExternalDataUtils.getDataSourceType(configuration).equals(DataSourceType.STREAM));
+ String nullString = configuration.get(ExternalDataConstants.KEY_NULL_STR);
+ return new DelimitedDataParser(ctx, valueParserFactories, delimiter, quote, hasHeader, recordType,
+ ExternalDataUtils.getDataSourceType(configuration).equals(DataSourceType.STREAM), nullString);
}
@Override
@@ -64,41 +63,7 @@
@Override
public IStreamDataParser createInputStreamParser(IHyracksTaskContext ctx, int partition)
throws HyracksDataException {
- return createParser();
- }
-
- // Get a delimiter from the given configuration
- public static char getDelimiter(Map<String, String> configuration) throws HyracksDataException {
- String delimiterValue = configuration.get(ExternalDataConstants.KEY_DELIMITER);
- if (delimiterValue == null) {
- delimiterValue = ExternalDataConstants.DEFAULT_DELIMITER;
- } else if (delimiterValue.length() != 1) {
- throw new RuntimeDataException(ErrorCode.PARSER_FACTORY_DELIMITED_DATA_PARSER_FACTORY_NOT_VALID_DELIMITER,
- delimiterValue);
- }
- return delimiterValue.charAt(0);
- }
-
- // Get a quote from the given configuration when the delimiter is given
- // Need to pass delimiter to check whether they share the same character
- public static char getQuote(Map<String, String> configuration, char delimiter) throws HyracksDataException {
- String quoteValue = configuration.get(ExternalDataConstants.KEY_QUOTE);
- if (quoteValue == null) {
- quoteValue = ExternalDataConstants.DEFAULT_QUOTE;
- } else if (quoteValue.length() != 1) {
- throw new RuntimeDataException(ErrorCode.PARSER_FACTORY_DELIMITED_DATA_PARSER_FACTORY_NOT_VALID_QUOTE,
- quoteValue);
- }
-
- // Since delimiter (char type value) can't be null,
- // we only check whether delimiter and quote use the same character
- if (quoteValue.charAt(0) == delimiter) {
- throw new RuntimeDataException(
- ErrorCode.PARSER_FACTORY_DELIMITED_DATA_PARSER_FACTORY_QUOTE_DELIMITER_MISMATCH, quoteValue,
- delimiter);
- }
-
- return quoteValue.charAt(0);
+ return createParser(ctx);
}
@Override
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/RecordWithMetadataParserFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/RecordWithMetadataParserFactory.java
index d34b5e0..3ae9b82 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/RecordWithMetadataParserFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/parser/factory/RecordWithMetadataParserFactory.java
@@ -23,6 +23,7 @@
import java.util.List;
import java.util.Map;
+import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.external.api.IRecordDataParser;
import org.apache.asterix.external.api.IRecordDataParserFactory;
@@ -54,12 +55,12 @@
// validate first
String recordFormat = configuration.get(ExternalDataConstants.KEY_RECORD_FORMAT);
if (recordFormat == null) {
- throw AlgebricksException.create(ErrorCode.UNKNOWN_RECORD_FORMAT_FOR_META_PARSER,
- ExternalDataConstants.KEY_FORMAT);
+ throw CompilationException.create(ErrorCode.UNKNOWN_RECORD_FORMAT_FOR_META_PARSER,
+ ExternalDataConstants.KEY_RECORD_FORMAT);
}
String format = configuration.get(ExternalDataConstants.KEY_FORMAT);
if (format == null) {
- throw AlgebricksException.create(ErrorCode.UNKNOWN_RECORD_FORMAT_FOR_META_PARSER,
+ throw CompilationException.create(ErrorCode.UNKNOWN_RECORD_FORMAT_FOR_META_PARSER,
ExternalDataConstants.KEY_FORMAT);
}
// Create Parser Factory
@@ -99,7 +100,7 @@
@Override
public IRecordDataParser<I> createRecordParser(IHyracksTaskContext ctx) throws HyracksDataException {
IRecordDataParser<O> recordParser = recordParserFactory.createRecordParser(ctx);
- return new RecordWithMetadataParser<I, O>(metaType, recordParser, converterFactory.createConverter());
+ return new RecordWithMetadataParser<I, O>(metaType, recordParser, converterFactory.createConverter(ctx));
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/AdapterFactoryProvider.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/AdapterFactoryProvider.java
index 5740143..96f06f4 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/AdapterFactoryProvider.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/AdapterFactoryProvider.java
@@ -21,56 +21,65 @@
import java.util.List;
import java.util.Map;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.asterix.external.adapter.factory.GenericAdapterFactory;
import org.apache.asterix.external.adapter.factory.LookupAdapterFactory;
-import org.apache.asterix.external.api.IAdapterFactory;
import org.apache.asterix.external.api.IIndexingAdapterFactory;
+import org.apache.asterix.external.api.ITypedAdapterFactory;
import org.apache.asterix.external.indexing.ExternalFile;
-import org.apache.asterix.external.util.ExternalDataCompatibilityUtils;
+import org.apache.asterix.external.util.ExternalDataUtils;
import org.apache.asterix.om.types.ARecordType;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
/**
* This class represents the entry point to all things adapters
*/
public class AdapterFactoryProvider {
- // Adapters
- public static IAdapterFactory getAdapterFactory(IServiceContext serviceCtx, String adapterName,
- Map<String, String> configuration, ARecordType itemType, ARecordType metaType)
- throws HyracksDataException, AlgebricksException {
- ExternalDataCompatibilityUtils.prepare(adapterName, configuration);
- GenericAdapterFactory adapterFactory = new GenericAdapterFactory();
+ private AdapterFactoryProvider() {
+ }
+
+ // get adapter factory. this method has the side effect of modifying the configuration as necessary
+ public static ITypedAdapterFactory getAdapterFactory(IServiceContext serviceCtx, String adapterName,
+ Map<String, String> configuration, ARecordType itemType, ARecordType metaType,
+ IWarningCollector warningCollector) throws HyracksDataException, AlgebricksException {
+ ExternalDataUtils.defaultConfiguration(configuration);
+ ExternalDataUtils.prepare(adapterName, configuration);
+ ICcApplicationContext context = (ICcApplicationContext) serviceCtx.getApplicationContext();
+ ITypedAdapterFactory adapterFactory =
+ (ITypedAdapterFactory) context.getAdapterFactoryService().createAdapterFactory();
adapterFactory.setOutputType(itemType);
adapterFactory.setMetaType(metaType);
- adapterFactory.configure(serviceCtx, configuration);
+ adapterFactory.configure(serviceCtx, configuration, warningCollector);
return adapterFactory;
}
- // Indexing Adapters
+ // get indexing adapter factory. this method has the side effect of modifying the configuration as necessary
public static IIndexingAdapterFactory getIndexingAdapterFactory(IServiceContext serviceCtx, String adapterName,
Map<String, String> configuration, ARecordType itemType, List<ExternalFile> snapshot, boolean indexingOp,
- ARecordType metaType) throws HyracksDataException, AlgebricksException {
- ExternalDataCompatibilityUtils.prepare(adapterName, configuration);
+ ARecordType metaType, IWarningCollector warningCollector) throws HyracksDataException, AlgebricksException {
+ ExternalDataUtils.defaultConfiguration(configuration);
+ ExternalDataUtils.prepare(adapterName, configuration);
GenericAdapterFactory adapterFactory = new GenericAdapterFactory();
adapterFactory.setOutputType(itemType);
adapterFactory.setMetaType(metaType);
adapterFactory.setSnapshot(snapshot, indexingOp);
- adapterFactory.configure(serviceCtx, configuration);
+ adapterFactory.configure(serviceCtx, configuration, warningCollector);
return adapterFactory;
}
// Lookup Adapters
public static LookupAdapterFactory<?> getLookupAdapterFactory(IServiceContext serviceCtx,
Map<String, String> configuration, ARecordType recordType, int[] ridFields, boolean retainInput,
- boolean retainMissing, IMissingWriterFactory missingWriterFactory)
+ boolean retainMissing, IMissingWriterFactory missingWriterFactory, IWarningCollector warningCollector)
throws HyracksDataException, AlgebricksException {
LookupAdapterFactory<?> adapterFactory =
new LookupAdapterFactory<>(recordType, ridFields, retainInput, retainMissing, missingWriterFactory);
- adapterFactory.configure(serviceCtx, configuration);
+ adapterFactory.configure(serviceCtx, configuration, warningCollector);
return adapterFactory;
}
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DataflowControllerProvider.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DataflowControllerProvider.java
index f9b012e..f60ecdc 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DataflowControllerProvider.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DataflowControllerProvider.java
@@ -54,6 +54,9 @@
public class DataflowControllerProvider {
+ private DataflowControllerProvider() {
+ }
+
// TODO: Instead, use a factory just like data source and data parser.
@SuppressWarnings({ "rawtypes", "unchecked" })
public static IDataFlowController getDataflowController(ARecordType recordType, IHyracksTaskContext ctx,
@@ -67,6 +70,8 @@
IRecordReader<?> recordReader = recordReaderFactory.createRecordReader(ctx, partition);
IRecordDataParserFactory<?> recordParserFactory = (IRecordDataParserFactory<?>) dataParserFactory;
IRecordDataParser<?> dataParser = recordParserFactory.createRecordParser(ctx);
+ // TODO(ali): revisit to think about passing data source name via setter or via createRecordParser
+ dataParser.configure(recordReader.getDataSourceName(), recordReader.getLineNumber());
if (indexingOp) {
return new IndexingDataFlowController(ctx, dataParser, recordReader,
((IIndexingDatasource) recordReader).getIndexer());
@@ -96,6 +101,7 @@
IInputStreamFactory streamFactory = (IInputStreamFactory) dataSourceFactory;
AsterixInputStream stream = streamFactory.createInputStream(ctx, partition);
IStreamDataParserFactory streamParserFactory = (IStreamDataParserFactory) dataParserFactory;
+ // TODO(ali): revisit to think about passing data source name to parser
IStreamDataParser streamParser = streamParserFactory.createInputStreamParser(ctx, partition);
streamParser.setInputStream(stream);
if (isFeed) {
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DatasourceFactoryProvider.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DatasourceFactoryProvider.java
index 8024dc4..2a2289c 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DatasourceFactoryProvider.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/DatasourceFactoryProvider.java
@@ -53,6 +53,7 @@
public static IExternalDataSourceFactory getExternalDataSourceFactory(ILibraryManager libraryManager,
Map<String, String> configuration) throws HyracksDataException, AsterixException {
+ // Take a copy of the configuration
if (ExternalDataUtils.getDataSourceType(configuration).equals(DataSourceType.RECORDS)) {
String reader = configuration.get(ExternalDataConstants.KEY_READER);
return DatasourceFactoryProvider.getRecordReaderFactory(libraryManager, reader, configuration);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/LookupReaderFactoryProvider.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/LookupReaderFactoryProvider.java
index 69f619f..db80c2a 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/LookupReaderFactoryProvider.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/LookupReaderFactoryProvider.java
@@ -26,18 +26,19 @@
import org.apache.asterix.external.util.ExternalDataConstants;
import org.apache.asterix.external.util.HDFSUtils;
import org.apache.hyracks.api.application.IServiceContext;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
public class LookupReaderFactoryProvider {
@SuppressWarnings("rawtypes")
public static ILookupReaderFactory getLookupReaderFactory(IServiceContext serviceCtx,
- Map<String, String> configuration) throws AsterixException {
+ Map<String, String> configuration, IWarningCollector warningCollector) throws AsterixException {
String inputFormat = HDFSUtils.getInputFormatClassName(configuration);
if (inputFormat.equals(ExternalDataConstants.CLASS_NAME_TEXT_INPUT_FORMAT)
|| inputFormat.equals(ExternalDataConstants.CLASS_NAME_SEQUENCE_INPUT_FORMAT)
|| inputFormat.equals(ExternalDataConstants.CLASS_NAME_RC_INPUT_FORMAT)) {
HDFSLookupReaderFactory<Object> readerFactory = new HDFSLookupReaderFactory<>();
- readerFactory.configure(serviceCtx, configuration);
+ readerFactory.configure(serviceCtx, configuration, warningCollector);
return readerFactory;
} else {
throw new AsterixException("Unrecognized external format");
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/ParserFactoryProvider.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/ParserFactoryProvider.java
index 53cf6b1..2265a25 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/ParserFactoryProvider.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/provider/ParserFactoryProvider.java
@@ -46,15 +46,12 @@
public static IDataParserFactory getDataParserFactory(ILibraryManager libraryManager,
Map<String, String> configuration) throws AsterixException {
IDataParserFactory parserFactory;
- String parserFactoryName = configuration.get(ExternalDataConstants.KEY_DATA_PARSER);
+ String parserFactoryName = configuration.get(ExternalDataConstants.KEY_PARSER);
if (ExternalDataUtils.isExternal(parserFactoryName)) {
return ExternalDataUtils.createExternalParserFactory(libraryManager,
ExternalDataUtils.getDataverse(configuration), parserFactoryName);
} else {
- String parserFactoryKey = ExternalDataUtils.getRecordFormat(configuration);
- if (parserFactoryKey == null) {
- parserFactoryKey = configuration.get(ExternalDataConstants.KEY_PARSER_FACTORY);
- }
+ String parserFactoryKey = ExternalDataUtils.getParserFactory(configuration);
parserFactory = ParserFactoryProvider.getDataParserFactory(parserFactoryKey);
}
return parserFactory;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataCompatibilityUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataCompatibilityUtils.java
index e222e99..8181262 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataCompatibilityUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataCompatibilityUtils.java
@@ -18,8 +18,6 @@
*/
package org.apache.asterix.external.util;
-import java.util.Map;
-
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.external.api.IDataParserFactory;
import org.apache.asterix.external.api.IExternalDataSourceFactory;
@@ -30,6 +28,9 @@
public class ExternalDataCompatibilityUtils {
+ private ExternalDataCompatibilityUtils() {
+ }
+
public static void validateCompatibility(IExternalDataSourceFactory dataSourceFactory,
IDataParserFactory dataParserFactory) throws AsterixException {
if (dataSourceFactory.getDataSourceType() != dataParserFactory.getDataSourceType()) {
@@ -58,16 +59,4 @@
+ recordParserFactory.getRecordClass());
}
}
-
- public static void prepare(String adapterName, Map<String, String> configuration) {
- if (!configuration.containsKey(ExternalDataConstants.KEY_READER)) {
- configuration.put(ExternalDataConstants.KEY_READER, adapterName);
- }
- if (!configuration.containsKey(ExternalDataConstants.KEY_PARSER)) {
- if (configuration.containsKey(ExternalDataConstants.KEY_FORMAT)) {
- configuration.put(ExternalDataConstants.KEY_PARSER,
- configuration.get(ExternalDataConstants.KEY_FORMAT));
- }
- }
- }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
index 729215e..265161e 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
@@ -18,7 +18,17 @@
*/
package org.apache.asterix.external.util;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.function.LongSupplier;
+import java.util.function.Supplier;
+
public class ExternalDataConstants {
+
+ private ExternalDataConstants() {
+ }
+
// TODO: Remove unused variables.
/**
* Keys
@@ -26,7 +36,7 @@
// used to specify the stream factory for an adapter that has a stream data source
public static final String KEY_STREAM = "stream";
// used to specify the dataverse of the adapter
- public static final String KEY_DATAVERSE = "dataverse";
+ public static final String KEY_DATASET_DATAVERSE = "dataset-dataverse";
// used to specify the socket addresses when reading data from sockets
public static final String KEY_SOCKETS = "sockets";
// specify whether the socket address points to an NC or an IP
@@ -61,7 +71,10 @@
public static final String KEY_EXPRESSION = "expression";
public static final String KEY_LOCAL_SOCKET_PATH = "local-socket-path";
public static final String KEY_FORMAT = "format";
+ public static final String KEY_INCLUDE = "include";
+ public static final String KEY_EXCLUDE = "exclude";
public static final String KEY_QUOTE = "quote";
+ public static final String KEY_ESCAPE = "escape";
public static final String KEY_PARSER = "parser";
public static final String KEY_DATASET_RECORD = "dataset-record";
public static final String KEY_HIVE_SERDE = "hive-serde";
@@ -71,7 +84,7 @@
public static final String KEY_WAIT_FOR_DATA = "wait-for-data";
public static final String KEY_FEED_NAME = "feed";
// a string representing external bucket name
- public static final String KEY_BUCKET = "bucket";
+ public static final String KEY_EXTERNAL_SOURCE_TYPE = "type";
// a comma delimited list of nodes
public static final String KEY_NODES = "nodes";
// a string representing the password used to authenticate with the external data source
@@ -101,6 +114,10 @@
public static final String KEY_HTTP_PROXY_PORT = "http-proxy-port";
public static final String KEY_HTTP_PROXY_USER = "http-proxy-user";
public static final String KEY_HTTP_PROXY_PASSWORD = "http-proxy-password";
+ // a string representing the NULL value
+ public static final String KEY_NULL_STR = "null";
+ public static final String KEY_REDACT_WARNINGS = "redact-warnings";
+ public static final String KEY_EXTERNAL_SCAN_BUFFER_SIZE = "external-scan-buffer-size";
/**
* Keys for adapter name
@@ -114,6 +131,7 @@
public static final String KEY_ADAPTER_NAME_SOCKET = "socket";
public static final String KEY_ALIAS_ADAPTER_NAME_SOCKET = "socket_adapter";
public static final String KEY_ADAPTER_NAME_HTTP = "http_adapter";
+ public static final String KEY_ADAPTER_NAME_AWS_S3 = "S3";
/**
* HDFS class names
@@ -162,6 +180,26 @@
public static final String FORMAT_LINE_SEPARATED = "line-separated";
public static final String FORMAT_HDFS_WRITABLE = "hdfs-writable";
public static final String FORMAT_KV = "kv";
+ public static final String FORMAT_CSV = "csv";
+ public static final String FORMAT_TSV = "tsv";
+ public static final Set<String> ALL_FORMATS;
+ static {
+ Set<String> formats = new HashSet<>(13);
+ formats.add(FORMAT_HIVE);
+ formats.add(FORMAT_BINARY);
+ formats.add(FORMAT_ADM);
+ formats.add(FORMAT_JSON_LOWER_CASE);
+ formats.add(FORMAT_DELIMITED_TEXT);
+ formats.add(FORMAT_TWEET);
+ formats.add(FORMAT_RSS);
+ formats.add(FORMAT_SEMISTRUCTURED);
+ formats.add(FORMAT_LINE_SEPARATED);
+ formats.add(FORMAT_HDFS_WRITABLE);
+ formats.add(FORMAT_KV);
+ formats.add(FORMAT_CSV);
+ formats.add(FORMAT_TSV);
+ ALL_FORMATS = Collections.unmodifiableSet(formats);
+ }
/**
* input streams
@@ -187,6 +225,8 @@
*/
public static final String TRUE = "true";
public static final String FALSE = "false";
+ public static final String TAB_STR = "\t";
+ public static final String NULL_STR = "\0";
/**
* Constant characters
@@ -199,6 +239,9 @@
public static final char CR = '\r';
public static final char DEFAULT_RECORD_START = '{';
public static final char DEFAULT_RECORD_END = '}';
+ public static final char OPEN_BRACKET = '[';
+ public static final char CLOSING_BRACKET = ']';
+ public static final char COMMA = ',';
/**
* Constant byte characters
@@ -213,6 +256,9 @@
public static final int DEFAULT_QUEUE_SIZE = 64;
public static final int MAX_RECORD_SIZE = 32000000;
+ public static final Supplier<String> EMPTY_STRING = () -> "";
+ public static final LongSupplier NO_LINES = () -> -1;
+
/**
* Expected parameter values
*/
@@ -226,7 +272,33 @@
public static final String EXTERNAL = "external";
public static final String KEY_READER_FACTORY = "reader-factory";
public static final String READER_RSS = "rss_feed";
- public static final String FORMAT_CSV = "csv";
public static final String ERROR_PARSE_RECORD = "Parser failed to parse record";
+ public static final String MISSING_FIELDS = "some fields are missing";
+ public static final String REC_ENDED_AT_EOF = "malformed input record ended abruptly";
+ public static final String EMPTY_FIELD = "empty value";
+ public static final String INVALID_VAL = "invalid value";
+
+ public static final String DEFINITION_FIELD_NAME = "definition";
+
+ public static class AwsS3 {
+ private AwsS3() {
+ throw new AssertionError("do not instantiate");
+ }
+
+ public static final String REGION_FIELD_NAME = "region";
+ public static final String ACCESS_KEY_ID_FIELD_NAME = "accessKeyId";
+ public static final String SECRET_ACCESS_KEY_FIELD_NAME = "secretAccessKey";
+ public static final String CONTAINER_NAME_FIELD_NAME = "container";
+ public static final String SERVICE_END_POINT_FIELD_NAME = "serviceEndpoint";
+
+ // AWS S3 specific error codes
+ public static final String ERROR_INTERNAL_ERROR = "InternalError";
+ public static final String ERROR_SLOW_DOWN = "SlowDown";
+ public static final String ERROR_METHOD_NOT_IMPLEMENTED = "NotImplemented";
+
+ public static boolean isRetryableError(String errorCode) {
+ return errorCode.equals(ERROR_INTERNAL_ERROR) || errorCode.equals(ERROR_SLOW_DOWN);
+ }
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
index a418cbf..e0b3562 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataUtils.java
@@ -18,12 +18,31 @@
*/
package org.apache.asterix.external.util;
+import static org.apache.asterix.common.exceptions.ErrorCode.REQUIRED_PARAM_IF_PARAM_IS_PRESENT;
+import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.ACCESS_KEY_ID_FIELD_NAME;
+import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.SECRET_ACCESS_KEY_FIELD_NAME;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_DELIMITER;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_ESCAPE;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_EXTERNAL_SCAN_BUFFER_SIZE;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_QUOTE;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_RECORD_END;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_RECORD_START;
+import static org.apache.asterix.runtime.evaluators.functions.StringEvaluatorUtils.RESERVED_REGEX_CHARS;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Arrays;
import java.util.EnumMap;
+import java.util.List;
import java.util.Map;
+import java.util.regex.Matcher;
import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.common.exceptions.WarningUtil;
import org.apache.asterix.common.library.ILibraryManager;
import org.apache.asterix.external.api.IDataParserFactory;
import org.apache.asterix.external.api.IExternalDataSourceFactory.DataSourceType;
@@ -32,8 +51,14 @@
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AUnionType;
+import org.apache.asterix.runtime.evaluators.common.NumberUtils;
import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.exceptions.Warning;
+import org.apache.hyracks.api.util.CleanupUtils;
+import org.apache.hyracks.dataflow.common.data.parsers.BooleanParserFactory;
import org.apache.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
import org.apache.hyracks.dataflow.common.data.parsers.FloatParserFactory;
import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory;
@@ -41,48 +66,64 @@
import org.apache.hyracks.dataflow.common.data.parsers.LongParserFactory;
import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
+import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.AwsBasicCredentials;
+import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;
+import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider;
+import software.amazon.awssdk.core.exception.SdkException;
+import software.amazon.awssdk.regions.Region;
+import software.amazon.awssdk.services.s3.S3Client;
+import software.amazon.awssdk.services.s3.S3ClientBuilder;
+import software.amazon.awssdk.services.s3.model.ListObjectsRequest;
+import software.amazon.awssdk.services.s3.model.ListObjectsResponse;
+import software.amazon.awssdk.services.s3.model.ListObjectsV2Request;
+import software.amazon.awssdk.services.s3.model.ListObjectsV2Response;
+import software.amazon.awssdk.services.s3.model.S3Exception;
+import software.amazon.awssdk.services.s3.model.S3Response;
+
public class ExternalDataUtils {
+ private static final Map<ATypeTag, IValueParserFactory> valueParserFactoryMap = new EnumMap<>(ATypeTag.class);
+ static {
+ valueParserFactoryMap.put(ATypeTag.INTEGER, IntegerParserFactory.INSTANCE);
+ valueParserFactoryMap.put(ATypeTag.FLOAT, FloatParserFactory.INSTANCE);
+ valueParserFactoryMap.put(ATypeTag.DOUBLE, DoubleParserFactory.INSTANCE);
+ valueParserFactoryMap.put(ATypeTag.BIGINT, LongParserFactory.INSTANCE);
+ valueParserFactoryMap.put(ATypeTag.STRING, UTF8StringParserFactory.INSTANCE);
+ valueParserFactoryMap.put(ATypeTag.BOOLEAN, BooleanParserFactory.INSTANCE);
+ }
+
+ private ExternalDataUtils() {
+ }
+
+ public static int getOrDefaultBufferSize(Map<String, String> configuration) {
+ String bufferSize = configuration.get(KEY_EXTERNAL_SCAN_BUFFER_SIZE);
+ return bufferSize != null ? Integer.parseInt(bufferSize) : ExternalDataConstants.DEFAULT_BUFFER_SIZE;
+ }
+
// Get a delimiter from the given configuration
- public static char getDelimiter(Map<String, String> configuration) throws AsterixException {
- String delimiterValue = configuration.get(ExternalDataConstants.KEY_DELIMITER);
- if (delimiterValue == null) {
- delimiterValue = ExternalDataConstants.DEFAULT_DELIMITER;
- } else if (delimiterValue.length() != 1) {
- throw new AsterixException(
- "'" + delimiterValue + "' is not a valid delimiter. The length of a delimiter should be 1.");
- }
- return delimiterValue.charAt(0);
+ public static char validateGetDelimiter(Map<String, String> configuration) throws HyracksDataException {
+ return validateCharOrDefault(configuration, KEY_DELIMITER, ExternalDataConstants.DEFAULT_DELIMITER.charAt(0));
}
// Get a quote from the given configuration when the delimiter is given
// Need to pass delimiter to check whether they share the same character
- public static char getQuote(Map<String, String> configuration, char delimiter) throws AsterixException {
- String quoteValue = configuration.get(ExternalDataConstants.KEY_QUOTE);
- if (quoteValue == null) {
- quoteValue = ExternalDataConstants.DEFAULT_QUOTE;
- } else if (quoteValue.length() != 1) {
- throw new AsterixException("'" + quoteValue + "' is not a valid quote. The length of a quote should be 1.");
- }
-
- // Since delimiter (char type value) can't be null,
- // we only check whether delimiter and quote use the same character
- if (quoteValue.charAt(0) == delimiter) {
- throw new AsterixException(
- "Quote '" + quoteValue + "' cannot be used with the delimiter '" + delimiter + "'. ");
- }
-
- return quoteValue.charAt(0);
+ public static char validateGetQuote(Map<String, String> configuration, char delimiter) throws HyracksDataException {
+ char quote = validateCharOrDefault(configuration, KEY_QUOTE, ExternalDataConstants.DEFAULT_QUOTE.charAt(0));
+ validateDelimiterAndQuote(delimiter, quote);
+ return quote;
}
- // Get the header flag
- public static boolean getHasHeader(Map<String, String> configuration) {
- return Boolean.parseBoolean(configuration.get(ExternalDataConstants.KEY_HEADER));
+ public static char validateGetEscape(Map<String, String> configuration) throws HyracksDataException {
+ return validateCharOrDefault(configuration, KEY_ESCAPE, ExternalDataConstants.ESCAPE);
}
- public static void validateParameters(Map<String, String> configuration) throws AsterixException {
- validateDataSourceParameters(configuration);
- validateDataParserParameters(configuration);
+ public static char validateGetRecordStart(Map<String, String> configuration) throws HyracksDataException {
+ return validateCharOrDefault(configuration, KEY_RECORD_START, ExternalDataConstants.DEFAULT_RECORD_START);
+ }
+
+ public static char validateGetRecordEnd(Map<String, String> configuration) throws HyracksDataException {
+ return validateCharOrDefault(configuration, KEY_RECORD_END, ExternalDataConstants.DEFAULT_RECORD_END);
}
public static void validateDataParserParameters(Map<String, String> configuration) throws AsterixException {
@@ -90,8 +131,8 @@
if (parser == null) {
String parserFactory = configuration.get(ExternalDataConstants.KEY_PARSER_FACTORY);
if (parserFactory == null) {
- throw new AsterixException("The parameter " + ExternalDataConstants.KEY_FORMAT + " or "
- + ExternalDataConstants.KEY_PARSER_FACTORY + " must be specified.");
+ throw AsterixException.create(ErrorCode.PARAMETERS_REQUIRED,
+ ExternalDataConstants.KEY_FORMAT + " or " + ExternalDataConstants.KEY_PARSER_FACTORY);
}
}
}
@@ -99,7 +140,7 @@
public static void validateDataSourceParameters(Map<String, String> configuration) throws AsterixException {
String reader = configuration.get(ExternalDataConstants.KEY_READER);
if (reader == null) {
- throw new AsterixException("The parameter " + ExternalDataConstants.KEY_READER + " must be specified.");
+ throw AsterixException.create(ErrorCode.PARAMETERS_REQUIRED, ExternalDataConstants.KEY_READER);
}
}
@@ -142,33 +183,16 @@
}
public static String getDataverse(Map<String, String> configuration) {
- return configuration.get(ExternalDataConstants.KEY_DATAVERSE);
+ return configuration.get(ExternalDataConstants.KEY_DATASET_DATAVERSE);
}
- public static String getRecordFormat(Map<String, String> configuration) {
- String parserFormat = configuration.get(ExternalDataConstants.KEY_DATA_PARSER);
- return parserFormat != null ? parserFormat : configuration.get(ExternalDataConstants.KEY_FORMAT);
- }
-
- public static void setRecordFormat(Map<String, String> configuration, String format) {
- if (!configuration.containsKey(ExternalDataConstants.KEY_DATA_PARSER)) {
- configuration.put(ExternalDataConstants.KEY_DATA_PARSER, format);
+ public static String getParserFactory(Map<String, String> configuration) {
+ String parserFactory = configuration.get(ExternalDataConstants.KEY_PARSER);
+ if (parserFactory != null) {
+ return parserFactory;
}
- if (!configuration.containsKey(ExternalDataConstants.KEY_FORMAT)) {
- configuration.put(ExternalDataConstants.KEY_FORMAT, format);
- }
- }
-
- private static Map<ATypeTag, IValueParserFactory> valueParserFactoryMap = initializeValueParserFactoryMap();
-
- private static Map<ATypeTag, IValueParserFactory> initializeValueParserFactoryMap() {
- Map<ATypeTag, IValueParserFactory> m = new EnumMap<>(ATypeTag.class);
- m.put(ATypeTag.INTEGER, IntegerParserFactory.INSTANCE);
- m.put(ATypeTag.FLOAT, FloatParserFactory.INSTANCE);
- m.put(ATypeTag.DOUBLE, DoubleParserFactory.INSTANCE);
- m.put(ATypeTag.BIGINT, LongParserFactory.INSTANCE);
- m.put(ATypeTag.STRING, UTF8StringParserFactory.INSTANCE);
- return m;
+ parserFactory = configuration.get(ExternalDataConstants.KEY_FORMAT);
+ return parserFactory != null ? parserFactory : configuration.get(ExternalDataConstants.KEY_PARSER_FACTORY);
}
public static IValueParserFactory[] getValueParserFactories(ARecordType recordType) {
@@ -201,16 +225,13 @@
return vpf;
}
- public static String getRecordReaderStreamName(Map<String, String> configuration) {
- return configuration.get(ExternalDataConstants.KEY_READER_STREAM);
+ public static boolean hasHeader(Map<String, String> configuration) {
+ return isTrue(configuration, ExternalDataConstants.KEY_HEADER);
}
- public static boolean hasHeader(Map<String, String> configuration) {
- String value = configuration.get(ExternalDataConstants.KEY_HEADER);
- if (value != null) {
- return Boolean.valueOf(value);
- }
- return false;
+ public static boolean isTrue(Map<String, String> configuration, String key) {
+ String value = configuration.get(key);
+ return value == null ? false : Boolean.valueOf(value);
}
public static IRecordReaderFactory<?> createExternalRecordReaderFactory(ILibraryManager libraryManager,
@@ -266,7 +287,7 @@
if (!configuration.containsKey(ExternalDataConstants.KEY_IS_FEED)) {
configuration.put(ExternalDataConstants.KEY_IS_FEED, ExternalDataConstants.TRUE);
}
- configuration.put(ExternalDataConstants.KEY_DATAVERSE, dataverseName);
+ configuration.put(ExternalDataConstants.KEY_DATASET_DATAVERSE, dataverseName);
configuration.put(ExternalDataConstants.KEY_FEED_NAME, feedName);
}
@@ -281,12 +302,6 @@
return configuration.get(ExternalDataConstants.KEY_FEED_NAME);
}
- public static int getQueueSize(Map<String, String> configuration) {
- return configuration.containsKey(ExternalDataConstants.KEY_QUEUE_SIZE)
- ? Integer.parseInt(configuration.get(ExternalDataConstants.KEY_QUEUE_SIZE))
- : ExternalDataConstants.DEFAULT_QUEUE_SIZE;
- }
-
public static boolean isRecordWithMeta(Map<String, String> configuration) {
return configuration.containsKey(ExternalDataConstants.KEY_META_TYPE_NAME);
}
@@ -306,8 +321,7 @@
public static int getNumberOfKeys(Map<String, String> configuration) throws AsterixException {
String keyIndexes = configuration.get(ExternalDataConstants.KEY_KEY_INDEXES);
if (keyIndexes == null) {
- throw new AsterixException(
- "A change feed must have the parameter " + ExternalDataConstants.KEY_KEY_INDEXES);
+ throw AsterixException.create(ErrorCode.PARAMETERS_REQUIRED, ExternalDataConstants.KEY_KEY_INDEXES);
}
return keyIndexes.split(",").length;
}
@@ -339,4 +353,463 @@
}
return intIndicators;
}
+
+ /**
+ * Fills the configuration of the external dataset and its adapter with default values if not provided by user.
+ *
+ * @param configuration external data configuration
+ */
+ public static void defaultConfiguration(Map<String, String> configuration) {
+ String format = configuration.get(ExternalDataConstants.KEY_FORMAT);
+ if (format != null) {
+ // default quote, escape character for quote and fields delimiter for csv and tsv format
+ if (format.equals(ExternalDataConstants.FORMAT_CSV)) {
+ configuration.putIfAbsent(KEY_DELIMITER, ExternalDataConstants.DEFAULT_DELIMITER);
+ configuration.putIfAbsent(KEY_QUOTE, ExternalDataConstants.DEFAULT_QUOTE);
+ configuration.putIfAbsent(KEY_ESCAPE, ExternalDataConstants.DEFAULT_QUOTE);
+ } else if (format.equals(ExternalDataConstants.FORMAT_TSV)) {
+ configuration.putIfAbsent(KEY_DELIMITER, ExternalDataConstants.TAB_STR);
+ configuration.putIfAbsent(KEY_QUOTE, ExternalDataConstants.NULL_STR);
+ configuration.putIfAbsent(KEY_ESCAPE, ExternalDataConstants.NULL_STR);
+ }
+ }
+ }
+
+ /**
+ * Prepares the configuration of the external data and its adapter by filling the information required by
+ * adapters and parsers.
+ *
+ * @param adapterName adapter name
+ * @param configuration external data configuration
+ */
+ public static void prepare(String adapterName, Map<String, String> configuration) {
+ if (!configuration.containsKey(ExternalDataConstants.KEY_READER)) {
+ configuration.put(ExternalDataConstants.KEY_READER, adapterName);
+ }
+ if (!configuration.containsKey(ExternalDataConstants.KEY_PARSER)
+ && configuration.containsKey(ExternalDataConstants.KEY_FORMAT)) {
+ configuration.put(ExternalDataConstants.KEY_PARSER, configuration.get(ExternalDataConstants.KEY_FORMAT));
+ }
+ }
+
+ /**
+ * Normalizes the values of certain parameters of the adapter configuration. This should happen before persisting
+ * the metadata (e.g. when creating external datasets or feeds) and when creating an adapter factory.
+ *
+ * @param configuration external data configuration
+ */
+ public static void normalize(Map<String, String> configuration) {
+ // normalize the "format" parameter
+ String paramValue = configuration.get(ExternalDataConstants.KEY_FORMAT);
+ if (paramValue != null) {
+ String lowerCaseFormat = paramValue.toLowerCase().trim();
+ if (ExternalDataConstants.ALL_FORMATS.contains(lowerCaseFormat)) {
+ configuration.put(ExternalDataConstants.KEY_FORMAT, lowerCaseFormat);
+ }
+ }
+ // normalize "header" parameter
+ putToLowerIfExists(configuration, ExternalDataConstants.KEY_HEADER);
+ // normalize "redact-warnings" parameter
+ putToLowerIfExists(configuration, ExternalDataConstants.KEY_REDACT_WARNINGS);
+ }
+
+ /**
+ * Validates the parameter values of the adapter configuration. This should happen after normalizing the values.
+ *
+ * @param configuration external data configuration
+ * @throws HyracksDataException HyracksDataException
+ */
+ public static void validate(Map<String, String> configuration) throws HyracksDataException {
+ String format = configuration.get(ExternalDataConstants.KEY_FORMAT);
+ String header = configuration.get(ExternalDataConstants.KEY_HEADER);
+ if (format != null && isHeaderRequiredFor(format) && header == null) {
+ throw new RuntimeDataException(ErrorCode.PARAMETERS_REQUIRED, ExternalDataConstants.KEY_HEADER);
+ }
+ if (header != null && !isBoolean(header)) {
+ throw new RuntimeDataException(ErrorCode.INVALID_REQ_PARAM_VAL, ExternalDataConstants.KEY_HEADER, header);
+ }
+ char delimiter = validateGetDelimiter(configuration);
+ validateGetQuote(configuration, delimiter);
+ validateGetEscape(configuration);
+ String value = configuration.get(ExternalDataConstants.KEY_REDACT_WARNINGS);
+ if (value != null && !isBoolean(value)) {
+ throw new RuntimeDataException(ErrorCode.INVALID_REQ_PARAM_VAL, ExternalDataConstants.KEY_REDACT_WARNINGS,
+ value);
+ }
+ }
+
+ private static boolean isHeaderRequiredFor(String format) {
+ return format.equals(ExternalDataConstants.FORMAT_CSV) || format.equals(ExternalDataConstants.FORMAT_TSV);
+ }
+
+ private static boolean isBoolean(String value) {
+ return value.equals(ExternalDataConstants.TRUE) || value.equals(ExternalDataConstants.FALSE);
+ }
+
+ private static void validateDelimiterAndQuote(char delimiter, char quote) throws RuntimeDataException {
+ if (quote == delimiter) {
+ throw new RuntimeDataException(ErrorCode.QUOTE_DELIMITER_MISMATCH, quote, delimiter);
+ }
+ }
+
+ private static char validateCharOrDefault(Map<String, String> configuration, String key, char defaultValue)
+ throws HyracksDataException {
+ String value = configuration.get(key);
+ if (value == null) {
+ return defaultValue;
+ }
+ validateChar(value, key);
+ return value.charAt(0);
+ }
+
+ public static void validateChar(String parameterValue, String parameterName) throws RuntimeDataException {
+ if (parameterValue.length() != 1) {
+ throw new RuntimeDataException(ErrorCode.INVALID_CHAR_LENGTH, parameterValue, parameterName);
+ }
+ }
+
+ private static void putToLowerIfExists(Map<String, String> configuration, String key) {
+ String paramValue = configuration.get(key);
+ if (paramValue != null) {
+ configuration.put(key, paramValue.toLowerCase().trim());
+ }
+ }
+
+ /**
+ * Validates adapter specific external dataset properties. Specific properties for different adapters should be
+ * validated here
+ *
+ * @param configuration properties
+ */
+ public static void validateAdapterSpecificProperties(Map<String, String> configuration, SourceLocation srcLoc,
+ IWarningCollector collector) throws CompilationException {
+ String type = configuration.get(ExternalDataConstants.KEY_EXTERNAL_SOURCE_TYPE);
+
+ switch (type) {
+ case ExternalDataConstants.KEY_ADAPTER_NAME_AWS_S3:
+ AwsS3.validateProperties(configuration, srcLoc, collector);
+ break;
+ default:
+ // Nothing needs to be done
+ break;
+ }
+ }
+
+ /**
+ * Regex matches all the provided patterns against the provided path
+ *
+ * @param path path to check against
+ *
+ * @return {@code true} if all patterns match, {@code false} otherwise
+ */
+ public static boolean matchPatterns(List<Matcher> matchers, String path) {
+ for (Matcher matcher : matchers) {
+ if (matcher.reset(path).matches()) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ /**
+ * Converts the wildcard to proper regex
+ *
+ * @param pattern wildcard pattern to convert
+ *
+ * @return regex expression
+ */
+ public static String patternToRegex(String pattern) {
+ int charPosition = 0;
+ int patternLength = pattern.length();
+ StringBuilder stuffBuilder = new StringBuilder();
+ StringBuilder result = new StringBuilder();
+
+ while (charPosition < patternLength) {
+ char c = pattern.charAt(charPosition);
+ charPosition++;
+
+ switch (c) {
+ case '*':
+ result.append(".*");
+ break;
+ case '?':
+ result.append(".");
+ break;
+ case '[':
+ int closingBracketPosition = charPosition;
+ if (closingBracketPosition < patternLength && pattern.charAt(closingBracketPosition) == '!') {
+ closingBracketPosition++;
+ }
+
+ // 2 cases can happen here:
+ // 1- Empty character class [] which is invalid for java, so treat ] as literal and find another
+ // closing bracket, if no closing bracket is found, the whole thing is a literal
+ // 2- Negated empty class [!] converted to [^] which is invalid for java, so treat ] as literal and
+ // find another closing bracket, if no closing bracket is found, the whole thing is a literal
+ if (closingBracketPosition < patternLength && pattern.charAt(closingBracketPosition) == ']') {
+ closingBracketPosition++;
+ }
+
+ // No [] and [!] cases, search for the closing bracket
+ while (closingBracketPosition < patternLength && pattern.charAt(closingBracketPosition) != ']') {
+ closingBracketPosition++;
+ }
+
+ // No closing bracket found (or [] or [!]), escape the opening bracket, treat it as literals
+ if (closingBracketPosition >= patternLength) {
+ result.append("\\[");
+ } else {
+ // Found closing bracket, get the stuff in between the found the character class ("[" and "]")
+ String stuff = pattern.substring(charPosition, closingBracketPosition);
+
+ stuffBuilder.setLength(0);
+ int stuffCharPos = 0;
+
+ // If first character in the character class is "!" then convert it to "^"
+ if (stuff.charAt(0) == '!') {
+ stuffBuilder.append('^');
+ stuffCharPos++; // ignore first character when escaping metacharacters next step
+ }
+
+ for (; stuffCharPos < stuff.length(); stuffCharPos++) {
+ char stuffChar = stuff.charAt(stuffCharPos);
+ if (stuffChar != '-' && Arrays.binarySearch(RESERVED_REGEX_CHARS, stuffChar) >= 0) {
+ stuffBuilder.append("\\");
+ }
+ stuffBuilder.append(stuffChar);
+ }
+
+ String stuffEscaped = stuffBuilder.toString();
+
+ // Escape the set operations
+ stuffEscaped = stuffEscaped.replace("&&", "\\&\\&").replace("~~", "\\~\\~")
+ .replace("||", "\\|\\|").replace("--", "\\-\\-");
+
+ result.append("[").append(stuffEscaped).append("]");
+ charPosition = closingBracketPosition + 1;
+ }
+ break;
+ default:
+ if (Arrays.binarySearch(RESERVED_REGEX_CHARS, c) >= 0) {
+ result.append("\\");
+ }
+ result.append(c);
+ break;
+ }
+ }
+
+ return result.toString();
+ }
+
+ /**
+ * Adjusts the prefix (if needed) and returns it
+ *
+ * @param configuration configuration
+ */
+ public static String getPrefix(Map<String, String> configuration) {
+ String definition = configuration.get(ExternalDataConstants.DEFINITION_FIELD_NAME);
+ if (definition != null && !definition.isEmpty()) {
+ return definition + (!definition.endsWith("/") ? "/" : "");
+ }
+ return "";
+ }
+
+ public static class AwsS3 {
+ private AwsS3() {
+ throw new AssertionError("do not instantiate");
+ }
+
+ /**
+ * Builds the S3 client using the provided configuration
+ *
+ * @param configuration properties
+ * @return S3 client
+ * @throws CompilationException CompilationException
+ */
+ public static S3Client buildAwsS3Client(Map<String, String> configuration) throws CompilationException {
+ // TODO(Hussain): Need to ensure that all required parameters are present in a previous step
+ String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
+ String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
+ String regionId = configuration.get(ExternalDataConstants.AwsS3.REGION_FIELD_NAME);
+ String serviceEndpoint = configuration.get(ExternalDataConstants.AwsS3.SERVICE_END_POINT_FIELD_NAME);
+
+ S3ClientBuilder builder = S3Client.builder();
+
+ // Credentials
+ AwsCredentialsProvider credentialsProvider;
+
+ // No auth required
+ if (accessKeyId == null) {
+ credentialsProvider = AnonymousCredentialsProvider.create();
+ } else {
+ credentialsProvider =
+ StaticCredentialsProvider.create(AwsBasicCredentials.create(accessKeyId, secretAccessKey));
+ }
+
+ builder.credentialsProvider(credentialsProvider);
+ builder.region(Region.of(regionId));
+
+ // Validate the service endpoint if present
+ if (serviceEndpoint != null) {
+ try {
+ URI uri = new URI(serviceEndpoint);
+ try {
+ builder.endpointOverride(uri);
+ } catch (NullPointerException ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex.getMessage());
+ }
+ } catch (URISyntaxException ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR,
+ String.format("Invalid service endpoint %s", serviceEndpoint));
+ }
+ }
+
+ return builder.build();
+ }
+
+ /**
+ * Validate external dataset properties
+ *
+ * @param configuration properties
+ * @throws CompilationException Compilation exception
+ */
+ public static void validateProperties(Map<String, String> configuration, SourceLocation srcLoc,
+ IWarningCollector collector) throws CompilationException {
+
+ // check if the format property is present
+ if (configuration.get(ExternalDataConstants.KEY_FORMAT) == null) {
+ throw new CompilationException(ErrorCode.PARAMETERS_REQUIRED, srcLoc, ExternalDataConstants.KEY_FORMAT);
+ }
+
+ // Both parameters should be passed, or neither should be passed (for anonymous/no auth)
+ String accessKeyId = configuration.get(ACCESS_KEY_ID_FIELD_NAME);
+ String secretAccessKey = configuration.get(SECRET_ACCESS_KEY_FIELD_NAME);
+ if (accessKeyId == null || secretAccessKey == null) {
+ // If one is passed, the other is required
+ if (accessKeyId != null) {
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, SECRET_ACCESS_KEY_FIELD_NAME,
+ ACCESS_KEY_ID_FIELD_NAME);
+ } else if (secretAccessKey != null) {
+ throw new CompilationException(REQUIRED_PARAM_IF_PARAM_IS_PRESENT, ACCESS_KEY_ID_FIELD_NAME,
+ SECRET_ACCESS_KEY_FIELD_NAME);
+ }
+ }
+
+ validateIncludeExclude(configuration);
+
+ // Check if the bucket is present
+ S3Client s3Client = buildAwsS3Client(configuration);;
+ S3Response response;
+ boolean useOldApi = false;
+ String container = configuration.get(ExternalDataConstants.AwsS3.CONTAINER_NAME_FIELD_NAME);
+ String prefix = getPrefix(configuration);
+
+ try {
+ response = isBucketEmpty(s3Client, container, prefix, false);
+ } catch (S3Exception ex) {
+ // Method not implemented, try falling back to old API
+ try {
+ // For error code, see https://docs.aws.amazon.com/AmazonS3/latest/API/ErrorResponses.html
+ if (ex.awsErrorDetails().errorCode()
+ .equals(ExternalDataConstants.AwsS3.ERROR_METHOD_NOT_IMPLEMENTED)) {
+ useOldApi = true;
+ response = isBucketEmpty(s3Client, container, prefix, true);
+ } else {
+ throw ex;
+ }
+ } catch (SdkException ex2) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex2.getMessage());
+ }
+ } catch (SdkException ex) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_ERROR, ex.getMessage());
+ } finally {
+ if (s3Client != null) {
+ CleanupUtils.close(s3Client, null);
+ }
+ }
+
+ boolean isEmpty = useOldApi ? ((ListObjectsResponse) response).contents().isEmpty()
+ : ((ListObjectsV2Response) response).contents().isEmpty();
+ if (isEmpty && collector.shouldWarn()) {
+ Warning warning =
+ WarningUtil.forAsterix(srcLoc, ErrorCode.EXTERNAL_SOURCE_CONFIGURATION_RETURNED_NO_FILES);
+ collector.warn(warning);
+ }
+
+ // Returns 200 only in case the bucket exists, otherwise, throws an exception. However, to
+ // ensure coverage, check if the result is successful as well and not only catch exceptions
+ if (!response.sdkHttpResponse().isSuccessful()) {
+ throw new CompilationException(ErrorCode.EXTERNAL_SOURCE_CONTAINER_NOT_FOUND, container);
+ }
+ }
+
+ /**
+ * Checks for a single object in the specified bucket to determine if the bucket is empty or not.
+ *
+ * @param s3Client s3 client
+ * @param container the container name
+ * @param prefix Prefix to be used
+ * @param useOldApi flag whether to use the old API or not
+ *
+ * @return returns the S3 response
+ */
+ private static S3Response isBucketEmpty(S3Client s3Client, String container, String prefix, boolean useOldApi) {
+ S3Response response;
+ if (useOldApi) {
+ ListObjectsRequest.Builder listObjectsBuilder = ListObjectsRequest.builder();
+ listObjectsBuilder.prefix(prefix);
+ response = s3Client.listObjects(listObjectsBuilder.bucket(container).maxKeys(1).build());
+ } else {
+ ListObjectsV2Request.Builder listObjectsBuilder = ListObjectsV2Request.builder();
+ listObjectsBuilder.prefix(prefix);
+ response = s3Client.listObjectsV2(listObjectsBuilder.bucket(container).maxKeys(1).build());
+ }
+ return response;
+ }
+
+ /**
+ * @param configuration
+ * @throws CompilationException
+ */
+ public static void validateIncludeExclude(Map<String, String> configuration) throws CompilationException {
+ // Ensure that include and exclude are not provided at the same time + ensure valid format or property
+ List<Map.Entry<String, String>> includes = new ArrayList<>();
+ List<Map.Entry<String, String>> excludes = new ArrayList<>();
+
+ // Accepted formats are include, include#1, include#2, ... etc, same for excludes
+ for (Map.Entry<String, String> entry : configuration.entrySet()) {
+ String key = entry.getKey();
+
+ if (key.equals(ExternalDataConstants.KEY_INCLUDE)) {
+ includes.add(entry);
+ } else if (key.equals(ExternalDataConstants.KEY_EXCLUDE)) {
+ excludes.add(entry);
+ } else if (key.startsWith(ExternalDataConstants.KEY_INCLUDE)
+ || key.startsWith(ExternalDataConstants.KEY_EXCLUDE)) {
+
+ // Split by the "#", length should be 2, left should be include/exclude, right should be integer
+ String[] splits = key.split("#");
+
+ if (key.startsWith(ExternalDataConstants.KEY_INCLUDE) && splits.length == 2
+ && splits[0].equals(ExternalDataConstants.KEY_INCLUDE)
+ && NumberUtils.isIntegerNumericString(splits[1])) {
+ includes.add(entry);
+ } else if (key.startsWith(ExternalDataConstants.KEY_EXCLUDE) && splits.length == 2
+ && splits[0].equals(ExternalDataConstants.KEY_EXCLUDE)
+ && NumberUtils.isIntegerNumericString(splits[1])) {
+ excludes.add(entry);
+ } else {
+ throw new CompilationException(ErrorCode.INVALID_PROPERTY_FORMAT, key);
+ }
+ }
+ }
+
+ // TODO: Should include/exclude be a common check or S3 specific?
+ // Ensure either include or exclude are provided, but not both of them
+ if (!includes.isEmpty() && !excludes.isEmpty()) {
+ throw new CompilationException(ErrorCode.PARAMETERS_NOT_ALLOWED_AT_SAME_TIME,
+ ExternalDataConstants.KEY_INCLUDE, ExternalDataConstants.KEY_EXCLUDE);
+ }
+ }
+ }
}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ParseUtil.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ParseUtil.java
new file mode 100644
index 0000000..7afc592
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ParseUtil.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.util;
+
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.Warning;
+
+public class ParseUtil {
+
+ private ParseUtil() {
+ }
+
+ public static void warn(IWarningCollector warningCollector, String dataSourceName, long lineNum, int fieldNum,
+ String warnMessage) {
+ warningCollector.warn(
+ Warning.forHyracks(null, ErrorCode.PARSING_ERROR, dataSourceName, lineNum, fieldNum, warnMessage));
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IRecordReaderFactory b/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IRecordReaderFactory
index 0d96658..fd3e473 100644
--- a/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IRecordReaderFactory
+++ b/asterixdb/asterix-external-data/src/main/resources/META-INF/services/org.apache.asterix.external.api.IRecordReaderFactory
@@ -20,3 +20,4 @@
org.apache.asterix.external.input.HDFSDataSourceFactory
org.apache.asterix.external.input.record.reader.stream.StreamRecordReaderFactory
org.apache.asterix.external.input.record.reader.http.HttpServerRecordReaderFactory
+org.apache.asterix.external.input.record.reader.aws.AwsS3ReaderFactory
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/test/ClassAdToADMTest.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/test/ClassAdToADMTest.java
index 47f784a..0bbb2b8 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/test/ClassAdToADMTest.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/classad/test/ClassAdToADMTest.java
@@ -48,9 +48,11 @@
import org.apache.commons.io.FileUtils;
import org.apache.hyracks.algebricks.data.IPrinter;
import org.apache.hyracks.algebricks.data.IPrinterFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.test.support.TestUtils;
import org.junit.Assert;
import junit.framework.Test;
@@ -58,6 +60,9 @@
import junit.framework.TestSuite;
public class ClassAdToADMTest extends TestCase {
+
+ private final IHyracksTaskContext ctx = TestUtils.createHyracksTask();
+
/**
* Create the test case
*
@@ -123,13 +128,14 @@
FileSystemWatcher watcher = new FileSystemWatcher(paths, null, false);
LocalFSInputStream in = new LocalFSInputStream(watcher);
SemiStructuredRecordReader recordReader = new SemiStructuredRecordReader();
- recordReader.configure(in, config);
+ recordReader.configure(ctx, in, config);
while (recordReader.hasNext()) {
tb.reset();
IRawRecord<char[]> record = recordReader.next();
- parser.parse(record, tb.getDataOutput());
- tb.addFieldEndOffset();
- printTuple(tb, printers, printStream);
+ if (parser.parse(record, tb.getDataOutput())) {
+ tb.addFieldEndOffset();
+ printTuple(tb, printers, printStream);
+ }
}
recordReader.close();
printStream.close();
@@ -162,7 +168,7 @@
FileSystemWatcher watcher = new FileSystemWatcher(paths, null, false);
LocalFSInputStream in = new LocalFSInputStream(watcher);
SemiStructuredRecordReader recordReader = new SemiStructuredRecordReader();
- recordReader.configure(in, config);
+ recordReader.configure(ctx, in, config);
try {
Value val = new Value(objectPool);
while (recordReader.hasNext()) {
@@ -204,7 +210,7 @@
FileSystemWatcher watcher = new FileSystemWatcher(paths, null, false);
LocalFSInputStream in = new LocalFSInputStream(watcher);
SemiStructuredRecordReader recordReader = new SemiStructuredRecordReader();
- recordReader.configure(in, config);
+ recordReader.configure(ctx, in, config);
try {
Value val = new Value(objectPool);
while (recordReader.hasNext()) {
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/awss3/AwsS3Test.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/awss3/AwsS3Test.java
new file mode 100644
index 0000000..f0a2223
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/awss3/AwsS3Test.java
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.external.input.awss3;
+
+import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.ERROR_INTERNAL_ERROR;
+import static org.apache.asterix.external.util.ExternalDataConstants.AwsS3.ERROR_SLOW_DOWN;
+
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.external.input.record.reader.aws.AwsS3InputStream;
+import org.apache.asterix.external.input.record.reader.aws.AwsS3InputStreamFactory;
+import org.apache.hyracks.api.exceptions.IFormattedException;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import software.amazon.awssdk.awscore.exception.AwsErrorDetails;
+import software.amazon.awssdk.services.s3.S3Client;
+import software.amazon.awssdk.services.s3.model.GetObjectRequest;
+import software.amazon.awssdk.services.s3.model.S3Exception;
+import software.amazon.awssdk.services.s3.model.S3Object;
+
+public class AwsS3Test {
+
+ @SuppressWarnings("unchecked")
+ @Test
+ public void testWorkloadDistribution() throws Exception {
+ AwsS3InputStreamFactory factory = new AwsS3InputStreamFactory();
+
+ List<S3Object> s3Objects = new ArrayList<>();
+ final int partitionsCount = 3;
+
+ // Create S3 objects, 9 objects, on 3 partitions, they should be 600 total size on each partition
+ S3Object.Builder builder = S3Object.builder();
+ s3Objects.add(builder.key("1.json").size(100L).build());
+ s3Objects.add(builder.key("2.json").size(100L).build());
+ s3Objects.add(builder.key("3.json").size(100L).build());
+ s3Objects.add(builder.key("4.json").size(200L).build());
+ s3Objects.add(builder.key("5.json").size(200L).build());
+ s3Objects.add(builder.key("6.json").size(200L).build());
+ s3Objects.add(builder.key("7.json").size(300L).build());
+ s3Objects.add(builder.key("8.json").size(300L).build());
+ s3Objects.add(builder.key("9.json").size(300L).build());
+
+ // invoke the distributeWorkLoad method
+ Method distributeWorkloadMethod =
+ AwsS3InputStreamFactory.class.getDeclaredMethod("distributeWorkLoad", List.class, int.class);
+ distributeWorkloadMethod.setAccessible(true);
+ distributeWorkloadMethod.invoke(factory, s3Objects, partitionsCount);
+
+ // get the partitionWorkLoadsBasedOnSize field and verify the result
+ Field distributeWorkloadField = AwsS3InputStreamFactory.class.getDeclaredField("partitionWorkLoadsBasedOnSize");
+ distributeWorkloadField.setAccessible(true);
+ List<AwsS3InputStreamFactory.PartitionWorkLoadBasedOnSize> workloads =
+ (List<AwsS3InputStreamFactory.PartitionWorkLoadBasedOnSize>) distributeWorkloadField.get(factory);
+
+ for (AwsS3InputStreamFactory.PartitionWorkLoadBasedOnSize workload : workloads) {
+ Assert.assertEquals(workload.getTotalSize(), 600);
+ }
+ }
+
+ @Test
+ public void s3InternalError() throws Exception {
+ // S3Client mock
+ S3Client s3ClientMock = Mockito.mock(S3Client.class);
+
+ // Prepare S3Exception with internal error code
+ AwsErrorDetails errorDetails = AwsErrorDetails.builder().errorCode(ERROR_INTERNAL_ERROR)
+ .errorMessage("Internal Error from AWS").build();
+ S3Exception internalErrorEx = (S3Exception) S3Exception.builder().awsErrorDetails(errorDetails).build();
+ Mockito.when(s3ClientMock.getObject(GetObjectRequest.builder().build())).thenThrow(internalErrorEx);
+
+ // Set S3Client mock
+ AwsS3InputStream inputStreamMock = Mockito.mock(AwsS3InputStream.class);
+ Field s3ClientField = AwsS3InputStream.class.getDeclaredField("s3Client");
+ s3ClientField.setAccessible(true);
+ s3ClientField.set(inputStreamMock, s3ClientMock);
+
+ // doGetInputStream method
+ Method doGetInputStreamMethod =
+ AwsS3InputStream.class.getDeclaredMethod("doGetInputStream", GetObjectRequest.class);
+ doGetInputStreamMethod.setAccessible(true);
+
+ try {
+ doGetInputStreamMethod.invoke(inputStreamMock, GetObjectRequest.builder().build());
+ } catch (Exception ex) {
+ Assert.assertTrue("Not internal error", ex.getCause() instanceof IFormattedException
+ && ex.getCause().toString().contains("ASX1108: External source error. Internal Error from AWS"));
+ }
+ }
+
+ @Test
+ public void s3SlowDown() throws Exception {
+ // S3Client mock
+ S3Client s3ClientMock = Mockito.mock(S3Client.class);
+
+ // Prepare S3Exception with slow down error code
+ AwsErrorDetails errorDetails =
+ AwsErrorDetails.builder().errorCode(ERROR_SLOW_DOWN).errorMessage("SlowDown Error from AWS").build();
+ S3Exception slowDownEx = (S3Exception) S3Exception.builder().awsErrorDetails(errorDetails).build();
+ Mockito.when(s3ClientMock.getObject(GetObjectRequest.builder().build())).thenThrow(slowDownEx);
+
+ // Set S3Client mock
+ AwsS3InputStream inputStreamMock = Mockito.mock(AwsS3InputStream.class);
+
+ // Set S3Client
+ Field s3ClientField = AwsS3InputStream.class.getDeclaredField("s3Client");
+ s3ClientField.setAccessible(true);
+ s3ClientField.set(inputStreamMock, s3ClientMock);
+
+ // doGetInputStream method
+ Method doGetInputStreamMethod =
+ AwsS3InputStream.class.getDeclaredMethod("doGetInputStream", GetObjectRequest.class);
+ doGetInputStreamMethod.setAccessible(true);
+
+ try {
+ doGetInputStreamMethod.invoke(inputStreamMock, GetObjectRequest.builder().build());
+ } catch (Exception ex) {
+ Assert.assertTrue("Not SlowDown error", ex.getCause() instanceof IFormattedException
+ && ex.getCause().toString().contains("ASX1108: External source error. SlowDown Error from AWS"));
+ }
+ }
+}
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/RecordWithPKTestReaderFactory.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/RecordWithPKTestReaderFactory.java
index 715ad02..c65e00d 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/RecordWithPKTestReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/RecordWithPKTestReaderFactory.java
@@ -32,6 +32,7 @@
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
public class RecordWithPKTestReaderFactory implements IRecordReaderFactory<RecordWithPK<char[]>> {
@@ -48,7 +49,8 @@
}
@Override
- public void configure(IServiceContext serviceCtx, final Map<String, String> configuration) {
+ public void configure(IServiceContext serviceCtx, final Map<String, String> configuration,
+ IWarningCollector warningCollector) {
this.serviceCtx = serviceCtx;
}
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/kv/KVTestReaderFactory.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/kv/KVTestReaderFactory.java
index c09b9eb..49ca23e 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/kv/KVTestReaderFactory.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/input/record/reader/kv/KVTestReaderFactory.java
@@ -29,6 +29,7 @@
import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
import com.couchbase.client.core.message.dcp.DCPRequest;
@@ -58,7 +59,8 @@
}
@Override
- public void configure(IServiceContext serviceCtx, final Map<String, String> configuration) {
+ public void configure(IServiceContext serviceCtx, final Map<String, String> configuration,
+ IWarningCollector warningCollector) {
this.serviceCtx = serviceCtx;
if (configuration.containsKey("num-of-records")) {
numOfRecords = Integer.parseInt(configuration.get("num-of-records"));
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/ClassAdParser.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/ClassAdParser.java
index 4a1c740..70116b3 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/ClassAdParser.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/ClassAdParser.java
@@ -290,7 +290,7 @@
return null;
}
- private void parseRecord(ARecordType recType, ClassAd pAd, DataOutput out) throws IOException, AsterixException {
+ private boolean parseRecord(ARecordType recType, ClassAd pAd, DataOutput out) throws IOException, AsterixException {
ArrayBackedValueStorage fieldValueBuffer = getTempBuffer();
ArrayBackedValueStorage fieldNameBuffer = getTempBuffer();
IARecordBuilder recBuilder = getRecordBuilder();
@@ -359,6 +359,7 @@
}
}
recBuilder.write(out, true);
+ return true;
}
private void writeFieldValueToBuffer(IAType fieldType, DataOutput out, String name, ExprTree tree, ClassAd pAd)
@@ -1742,7 +1743,7 @@
}
@Override
- public void parse(IRawRecord<? extends char[]> record, DataOutput out) throws HyracksDataException {
+ public boolean parse(IRawRecord<? extends char[]> record, DataOutput out) throws HyracksDataException {
try {
resetPools();
if (oldFormat) {
@@ -1768,7 +1769,7 @@
rootAd.reset();
asterixParseClassAd(rootAd);
}
- parseRecord(recordType, rootAd, out);
+ return parseRecord(recordType, rootAd, out);
} catch (Exception e) {
throw HyracksDataException.create(e);
}
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapterFactory.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapterFactory.java
index 8ee8a57..0bd601b 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapterFactory.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/library/adapter/TestTypedAdapterFactory.java
@@ -25,9 +25,9 @@
import org.apache.asterix.common.api.IApplicationContext;
import org.apache.asterix.common.cluster.ClusterPartition;
import org.apache.asterix.common.dataflow.ICcApplicationContext;
-import org.apache.asterix.external.api.IAdapterFactory;
-import org.apache.asterix.external.api.IDataSourceAdapter;
+import org.apache.asterix.common.external.IDataSourceAdapter;
import org.apache.asterix.external.api.IExternalDataSourceFactory;
+import org.apache.asterix.external.api.ITypedAdapterFactory;
import org.apache.asterix.external.dataflow.TupleForwarder;
import org.apache.asterix.external.parser.ADMDataParser;
import org.apache.asterix.om.types.ARecordType;
@@ -37,11 +37,12 @@
import org.apache.hyracks.api.comm.IFrameWriter;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import org.apache.hyracks.dataflow.std.file.ITupleParser;
import org.apache.hyracks.dataflow.std.file.ITupleParserFactory;
-public class TestTypedAdapterFactory implements IAdapterFactory {
+public class TestTypedAdapterFactory implements ITypedAdapterFactory {
private static final long serialVersionUID = 1L;
@@ -113,7 +114,8 @@
}
@Override
- public void configure(IServiceContext serviceContext, Map<String, String> configuration) {
+ public void configure(IServiceContext serviceContext, Map<String, String> configuration,
+ IWarningCollector warningCollector) {
this.serviceContext = serviceContext;
this.configuration = configuration;
}
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/TestRecordWithPKParser.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/TestRecordWithPKParser.java
index dad9cfd..925eeee 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/TestRecordWithPKParser.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/TestRecordWithPKParser.java
@@ -36,9 +36,9 @@
}
@Override
- public void parse(final IRawRecord<? extends RecordWithPK<T>> record, final DataOutput out)
+ public boolean parse(final IRawRecord<? extends RecordWithPK<T>> record, final DataOutput out)
throws HyracksDataException {
- recordParser.parse(record.get().getRecord(), out);
+ return recordParser.parse(record.get().getRecord(), out);
}
@Override
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/ByteBufUTF8DecodeTest.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/ByteBufUTF8DecodeTest.java
index 2ba5a3e..2972542 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/ByteBufUTF8DecodeTest.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/ByteBufUTF8DecodeTest.java
@@ -41,6 +41,8 @@
import org.apache.asterix.external.input.stream.LocalFSInputStream;
import org.apache.asterix.external.util.ExternalDataConstants;
import org.apache.asterix.external.util.FileSystemWatcher;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.test.support.TestUtils;
import org.junit.Assert;
import org.junit.Test;
@@ -56,6 +58,7 @@
private final CharBuffer chars = CharBuffer.allocate(BUFFER_SIZE);
private final CharArrayRecord value = new CharArrayRecord();
private final ByteBuf nettyBuffer = UnpooledByteBufAllocator.DEFAULT.heapBuffer(KB32, Integer.MAX_VALUE);
+ private final IHyracksTaskContext ctx = TestUtils.createHyracksTask();
@Test
public void eatGlass() {
@@ -83,7 +86,7 @@
FileSystemWatcher watcher = new FileSystemWatcher(paths, null, false);
LocalFSInputStream in = new LocalFSInputStream(watcher);
try (SemiStructuredRecordReader recordReader = new SemiStructuredRecordReader()) {
- recordReader.configure(in, config);
+ recordReader.configure(ctx, in, config);
while (recordReader.hasNext()) {
try {
IRawRecord<char[]> record = recordReader.next();
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/RecordWithMetaTest.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/RecordWithMetaTest.java
index 1584065..888de08 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/RecordWithMetaTest.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/RecordWithMetaTest.java
@@ -44,12 +44,15 @@
import org.apache.commons.io.FileUtils;
import org.apache.hyracks.algebricks.data.IPrinter;
import org.apache.hyracks.algebricks.data.IPrinterFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.test.support.TestUtils;
import org.junit.Assert;
public class RecordWithMetaTest {
+ private final IHyracksTaskContext ctx = TestUtils.createHyracksTask();
private static ARecordType recordType;
@SuppressWarnings({ "unchecked", "rawtypes" })
@@ -90,10 +93,11 @@
config.put(ExternalDataConstants.KEY_HEADER, "true");
config.put(ExternalDataConstants.KEY_QUOTE, ExternalDataConstants.DEFAULT_QUOTE);
LineRecordReader lineReader = new LineRecordReader();
- lineReader.configure(inputStream, config);
+ lineReader.configure(ctx, inputStream, config);
// create csv with json record reader
- CSVToRecordWithMetadataAndPKConverter recordConverter = new CSVToRecordWithMetadataAndPKConverter(
- valueIndex, delimiter, metaType, recordType, pkIndicators, pkIndexes, keyTypes);
+ CSVToRecordWithMetadataAndPKConverter recordConverter =
+ new CSVToRecordWithMetadataAndPKConverter(valueIndex, delimiter, metaType, recordType, pkIndicators,
+ pkIndexes, keyTypes, ctx.getWarningCollector());
// create the value parser <ADM in this case>
ADMDataParser valueParser = new ADMDataParser(recordType, false);
// create parser.
@@ -121,14 +125,14 @@
while (lineReader.hasNext()) {
IRawRecord<char[]> record = lineReader.next();
tb.reset();
- parser.parse(record, tb.getDataOutput());
- tb.addFieldEndOffset();
- parser.parseMeta(tb.getDataOutput());
- tb.addFieldEndOffset();
- parser.appendLastParsedPrimaryKeyToTuple(tb);
- //print tuple
- printTuple(tb, printers, printStream);
-
+ if (parser.parse(record, tb.getDataOutput())) {
+ tb.addFieldEndOffset();
+ parser.parseMeta(tb.getDataOutput());
+ tb.addFieldEndOffset();
+ parser.appendLastParsedPrimaryKeyToTuple(tb);
+ //print tuple
+ printTuple(tb, printers, printStream);
+ }
}
lineReader.close();
printStream.close();
diff --git a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/TweetParserTest.java b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/TweetParserTest.java
index e183a84..b39ca45 100644
--- a/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/TweetParserTest.java
+++ b/asterixdb/asterix-external-data/src/test/java/org/apache/asterix/external/parser/test/TweetParserTest.java
@@ -72,7 +72,9 @@
for (int iter1 = 0; iter1 < lines.size(); iter1++) {
record.set(lines.get(iter1));
try {
- parser.parse(record, output);
+ if (!parser.parse(record, output)) {
+ Assert.fail("Unexpected failure in parser.");
+ }
} catch (HyracksDataException e) {
e.printStackTrace();
Assert.fail("Unexpected failure in parser.");
@@ -98,8 +100,9 @@
for (int iter1 = 0; iter1 < lines.size(); iter1++) {
record.set(lines.get(iter1));
try {
- parser.parse(record, output);
- regularCount++;
+ if (parser.parse(record, output)) {
+ regularCount++;
+ }
} catch (HyracksDataException e) {
Assert.assertTrue(e.toString().contains("Non-null") && (iter1 == 0 || iter1 == 1));
}
diff --git a/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj b/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
index 4c8820f..4bf4ea4 100644
--- a/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
+++ b/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
@@ -476,7 +476,7 @@
DatasetDecl dsetDecl = null;
boolean autogenerated = false;
Pair<Integer, List<String>> filterField = null;
- Pair<Identifier,Identifier> metaTypeComponents = new Pair<Identifier, Identifier>(null, null);
+ Pair<Identifier,Identifier> metaTypeComponents = null;
RecordConstructor withRecord = null;
}
{
@@ -495,10 +495,8 @@
try{
dsetDecl = new DatasetDecl(nameComponents.first,
nameComponents.second,
- typeComponents.first,
- typeComponents.second,
- metaTypeComponents.first,
- metaTypeComponents.second,
+ new TypeReferenceExpression(typeComponents),
+ null,
nodeGroupName != null? new Identifier(nodeGroupName): null,
hints,
DatasetType.EXTERNAL,
@@ -542,10 +540,8 @@
try{
dsetDecl = new DatasetDecl(nameComponents.first,
nameComponents.second,
- typeComponents.first,
- typeComponents.second,
- metaTypeComponents.first,
- metaTypeComponents.second,
+ new TypeReferenceExpression(typeComponents),
+ metaTypeComponents != null ? new TypeReferenceExpression(metaTypeComponents) : null,
nodeGroupName != null ? new Identifier(nodeGroupName) : null,
hints,
DatasetType.INTERNAL,
@@ -1242,7 +1238,9 @@
String value;
}
{
- <LEFTPAREN> key = StringLiteral() <EQ> value = StringLiteral() <RIGHTPAREN>
+ <LEFTPAREN> key = StringLiteral()
+ <EQ>( value = StringLiteral() | (<TRUE> | <FALSE>) {value = token.image.toLowerCase();} )
+ <RIGHTPAREN>
{
return new Pair<String, String>(key, value);
}
@@ -1366,7 +1364,7 @@
{
String fieldName;
TypeExpression type = null;
- boolean nullable = false;
+ boolean nullable = false, missable = false;
}
{
fieldName = Identifier()
@@ -1374,9 +1372,9 @@
String hint = getHint(token);
IRecordFieldDataGen rfdg = hint != null ? parseFieldDataGen(hint) : null;
}
- <COLON> type = TypeExpr() (<QUES> { nullable = true; } )?
+ <COLON> type = TypeExpr() (<QUES> { nullable = true; missable = true; } )?
{
- recType.addField(fieldName, type, nullable, rfdg);
+ recType.addField(fieldName, type, nullable, missable, rfdg);
}
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/RecordTypeDefinition.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/RecordTypeDefinition.java
index 4fcc7ae..8cdd911 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/RecordTypeDefinition.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/expression/RecordTypeDefinition.java
@@ -37,7 +37,8 @@
private final List<String> fieldNames = new ArrayList<>();
private final List<TypeExpression> fieldTypes = new ArrayList<>();
private final List<IRecordFieldDataGen> fieldDataGen = new ArrayList<>();
- private final List<Boolean> optionalFields = new ArrayList<>();
+ private final List<Boolean> nullableFields = new ArrayList<>();
+ private final List<Boolean> missableFields = new ArrayList<>();
private RecordKind recordKind;
private UndeclaredFieldsDataGen undeclaredFieldsDataGen;
@@ -50,17 +51,20 @@
return TypeExprKind.RECORD;
}
- public void addField(String name, TypeExpression type, Boolean nullable, IRecordFieldDataGen fldDataGen) {
+ public void addField(String name, TypeExpression type, boolean nullable, boolean missable,
+ IRecordFieldDataGen fldDataGen) {
fieldNames.add(name);
fieldTypes.add(type);
- optionalFields.add(nullable);
+ nullableFields.add(nullable);
+ missableFields.add(missable);
fieldDataGen.add(fldDataGen);
}
- public void addField(String name, TypeExpression type, Boolean optional) {
+ public void addField(String name, TypeExpression type, boolean nullable, boolean missable) {
fieldNames.add(name);
fieldTypes.add(type);
- optionalFields.add(optional);
+ nullableFields.add(nullable);
+ missableFields.add(missable);
}
public List<String> getFieldNames() {
@@ -71,8 +75,12 @@
return fieldTypes;
}
- public List<Boolean> getOptionableFields() {
- return optionalFields;
+ public List<Boolean> getNullableFields() {
+ return nullableFields;
+ }
+
+ public List<Boolean> getMissableFields() {
+ return missableFields;
}
public List<IRecordFieldDataGen> getFieldDataGen() {
@@ -102,7 +110,8 @@
@Override
public int hashCode() {
- return Objects.hash(fieldDataGen, fieldNames, fieldTypes, optionalFields, recordKind, undeclaredFieldsDataGen);
+ return Objects.hash(fieldDataGen, fieldNames, fieldTypes, nullableFields, missableFields, recordKind,
+ undeclaredFieldsDataGen);
}
@Override
@@ -116,7 +125,8 @@
}
RecordTypeDefinition target = (RecordTypeDefinition) object;
return fieldDataGen.equals(target.getFieldDataGen()) && fieldNames.equals(target.getFieldNames())
- && fieldTypes.equals(target.getFieldTypes()) && optionalFields.equals(target.getOptionableFields())
+ && fieldTypes.equals(target.getFieldTypes()) && nullableFields.equals(target.getNullableFields())
+ && missableFields.equals(target.getMissableFields())
&& Objects.equals(recordKind, target.getRecordKind())
&& Objects.equals(undeclaredFieldsDataGen, target.getUndeclaredFieldsDataGen());
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java
index 0a17b24..22753d0 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/DatasetDecl.java
@@ -25,6 +25,7 @@
import org.apache.asterix.lang.common.base.AbstractStatement;
import org.apache.asterix.lang.common.base.Statement;
import org.apache.asterix.lang.common.expression.RecordConstructor;
+import org.apache.asterix.lang.common.expression.TypeExpression;
import org.apache.asterix.lang.common.struct.Identifier;
import org.apache.asterix.lang.common.util.ConfigurationUtil;
import org.apache.asterix.lang.common.util.DatasetDeclParametersUtil;
@@ -36,38 +37,25 @@
public class DatasetDecl extends AbstractStatement {
protected final Identifier name;
protected final Identifier dataverse;
- protected final Identifier itemTypeDataverse;
- protected final Identifier itemTypeName;
- protected final Identifier metaItemTypeDataverse;
- protected final Identifier metaItemTypeName;
+ protected final TypeExpression itemType;
+ protected final TypeExpression metaItemType;
protected final Identifier nodegroupName;
protected final DatasetType datasetType;
protected final IDatasetDetailsDecl datasetDetailsDecl;
protected final Map<String, String> hints;
- private final AdmObjectNode withObjectNode;
+ private AdmObjectNode withObjectNode;
protected final boolean ifNotExists;
- public DatasetDecl(Identifier dataverse, Identifier name, Identifier itemTypeDataverse, Identifier itemTypeName,
- Identifier metaItemTypeDataverse, Identifier metaItemTypeName, Identifier nodeGroupName,
- Map<String, String> hints, DatasetType datasetType, IDatasetDetailsDecl idd, RecordConstructor withRecord,
- boolean ifNotExists) throws CompilationException {
+ public DatasetDecl(Identifier dataverse, Identifier name, TypeExpression itemType, TypeExpression metaItemType,
+ Identifier nodeGroupName, Map<String, String> hints, DatasetType datasetType, IDatasetDetailsDecl idd,
+ RecordConstructor withRecord, boolean ifNotExists) throws CompilationException {
this.dataverse = dataverse;
this.name = name;
- this.itemTypeName = itemTypeName;
- if (itemTypeDataverse.getValue() == null) {
- this.itemTypeDataverse = dataverse;
- } else {
- this.itemTypeDataverse = itemTypeDataverse;
- }
- this.metaItemTypeName = metaItemTypeName;
- if (metaItemTypeDataverse == null || metaItemTypeDataverse.getValue() == null) {
- this.metaItemTypeDataverse = dataverse;
- } else {
- this.metaItemTypeDataverse = metaItemTypeDataverse;
- }
+ this.itemType = itemType;
+ this.metaItemType = metaItemType;
this.nodegroupName = nodeGroupName;
this.hints = hints;
- this.withObjectNode = DatasetDeclParametersUtil.validateAndGetWithObjectNode(withRecord);
+ this.withObjectNode = DatasetDeclParametersUtil.validateAndGetWithObjectNode(withRecord, datasetType);
this.ifNotExists = ifNotExists;
this.datasetType = datasetType;
this.datasetDetailsDecl = idd;
@@ -85,40 +73,12 @@
return name;
}
- public Identifier getItemTypeName() {
- return itemTypeName;
+ public TypeExpression getItemType() {
+ return itemType;
}
- public Identifier getItemTypeDataverse() {
- return itemTypeDataverse;
- }
-
- public String getQualifiedTypeName() {
- if (itemTypeDataverse == dataverse) {
- return itemTypeName.getValue();
- } else {
- return itemTypeDataverse.getValue() + "." + itemTypeName.getValue();
- }
- }
-
- public Identifier getMetaName() {
- return name;
- }
-
- public Identifier getMetaItemTypeName() {
- return metaItemTypeName == null ? new Identifier() : metaItemTypeName;
- }
-
- public Identifier getMetaItemTypeDataverse() {
- return metaItemTypeDataverse == null ? new Identifier() : metaItemTypeDataverse;
- }
-
- public String getQualifiedMetaTypeName() {
- if (metaItemTypeDataverse == dataverse) {
- return metaItemTypeName.getValue();
- } else {
- return metaItemTypeDataverse.getValue() + "." + metaItemTypeName.getValue();
- }
+ public TypeExpression getMetaItemType() {
+ return metaItemType;
}
public Identifier getNodegroupName() {
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/DatasetDeclParametersUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/DatasetDeclParametersUtil.java
index a26a638..52285d9 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/DatasetDeclParametersUtil.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/DatasetDeclParametersUtil.java
@@ -18,6 +18,7 @@
*/
package org.apache.asterix.lang.common.util;
+import org.apache.asterix.common.config.DatasetConfig;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.lang.common.expression.RecordConstructor;
import org.apache.asterix.object.base.AdmObjectNode;
@@ -60,14 +61,21 @@
private DatasetDeclParametersUtil() {
}
- public static AdmObjectNode validateAndGetWithObjectNode(RecordConstructor withRecord) throws CompilationException {
+ public static AdmObjectNode validateAndGetWithObjectNode(RecordConstructor withRecord,
+ DatasetConfig.DatasetType datasetType) throws CompilationException {
if (withRecord == null) {
return EMPTY_WITH_OBJECT;
}
- final ConfigurationTypeValidator validator = new ConfigurationTypeValidator();
- final AdmObjectNode node = ExpressionUtils.toNode(withRecord);
- validator.validateType(WITH_OBJECT_TYPE, node);
- return node;
+
+ // Handle based on dataset type
+ if (datasetType == DatasetConfig.DatasetType.INTERNAL) {
+ final ConfigurationTypeValidator validator = new ConfigurationTypeValidator();
+ final AdmObjectNode node = ExpressionUtils.toNode(withRecord);
+ validator.validateType(WITH_OBJECT_TYPE, node);
+ return node;
+ } else {
+ return ExpressionUtils.toNode(withRecord);
+ }
}
private static ARecordType getWithObjectType() {
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/LangRecordParseUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/LangRecordParseUtil.java
index 6a34f65..28639c1 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/LangRecordParseUtil.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/LangRecordParseUtil.java
@@ -25,6 +25,7 @@
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.lang.common.base.Expression;
import org.apache.asterix.lang.common.base.Literal;
import org.apache.asterix.lang.common.expression.FieldBinding;
@@ -69,8 +70,8 @@
case LIST_CONSTRUCTOR_EXPRESSION:
return parseList((ListConstructor) expr);
default:
- throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.PARSE_ERROR,
- NOT_ALLOWED_EXPRESSIONS_ERROR_MESSAGE, Expression.Kind.LITERAL_EXPRESSION.toString(),
+ throw new RuntimeDataException(ErrorCode.PARSE_ERROR, NOT_ALLOWED_EXPRESSIONS_ERROR_MESSAGE,
+ Expression.Kind.LITERAL_EXPRESSION.toString(),
Expression.Kind.RECORD_CONSTRUCTOR_EXPRESSION.toString(),
Expression.Kind.LIST_CONSTRUCTOR_EXPRESSION.toString());
}
@@ -110,7 +111,7 @@
private static AdmArrayNode parseList(ListConstructor valueExpr) throws CompilationException, HyracksDataException {
if (valueExpr.getType() != ListConstructor.Type.ORDERED_LIST_CONSTRUCTOR) {
- throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.PARSE_ERROR, "JSON List can't be of type %1$s",
+ throw new RuntimeDataException(ErrorCode.PARSE_ERROR, "JSON List can't be of type %1$s",
valueExpr.getType());
}
List<Expression> exprs = valueExpr.getExprList();
@@ -140,7 +141,7 @@
case STRING:
return new AdmStringNode((String) value.getValue());
default:
- throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.PARSE_ERROR, "Unknown Literal Type %1$s",
+ throw new RuntimeDataException(ErrorCode.PARSE_ERROR, "Unknown Literal Type %1$s",
value.getLiteralType());
}
}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
index 6b734dd..28a7be2 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/FormatPrintVisitor.java
@@ -415,7 +415,8 @@
out.println("{");
Iterator<String> nameIter = r.getFieldNames().iterator();
Iterator<TypeExpression> typeIter = r.getFieldTypes().iterator();
- Iterator<Boolean> isOptionalIter = r.getOptionableFields().iterator();
+ Iterator<Boolean> isNullableIter = r.getNullableFields().iterator();
+ Iterator<Boolean> isMissableIter = r.getMissableFields().iterator();
boolean first = true;
while (nameIter.hasNext()) {
if (first) {
@@ -425,10 +426,11 @@
}
String name = normalize(nameIter.next());
TypeExpression texp = typeIter.next();
- Boolean isNullable = isOptionalIter.next();
+ Boolean isNullable = isNullableIter.next();
+ Boolean isMissable = isMissableIter.next();
out.print(skip(step) + name + " : ");
texp.accept(this, step + 2);
- if (isNullable) {
+ if (isNullable || isMissable) {
out.print("?");
}
}
@@ -457,16 +459,18 @@
public Void visit(DatasetDecl dd, Integer step) throws CompilationException {
if (dd.getDatasetType() == DatasetType.INTERNAL) {
out.print(skip(step) + "create " + datasetSymbol + generateFullName(dd.getDataverse(), dd.getName())
- + generateIfNotExists(dd.getIfNotExists()) + "(" + dd.getQualifiedTypeName() + ")"
- + " primary key ");
+ + generateIfNotExists(dd.getIfNotExists()) + "(");
+ dd.getItemType().accept(this, step + 2);
+ out.print(skip(step) + ") primary key ");
printDelimitedKeys(((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getPartitioningExprs(), ",");
if (((InternalDetailsDecl) dd.getDatasetDetailsDecl()).isAutogenerated()) {
out.print(" autogenerated ");
}
} else if (dd.getDatasetType() == DatasetType.EXTERNAL) {
- out.print(
- skip(step) + "create external " + datasetSymbol + generateFullName(dd.getDataverse(), dd.getName())
- + "(" + dd.getQualifiedTypeName() + ")" + generateIfNotExists(dd.getIfNotExists()));
+ out.print(skip(step) + "create external " + datasetSymbol
+ + generateFullName(dd.getDataverse(), dd.getName()) + "(");
+ dd.getItemType().accept(this, step + 2);
+ out.print(skip(step) + ")" + generateIfNotExists(dd.getIfNotExists()));
ExternalDetailsDecl externalDetails = (ExternalDetailsDecl) dd.getDatasetDetailsDecl();
out.print(" using " + revertStringToQuoted(externalDetails.getAdapter()));
printConfiguration(externalDetails.getProperties());
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/QueryPrintVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/QueryPrintVisitor.java
index ff55880..0828aba 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/QueryPrintVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/QueryPrintVisitor.java
@@ -347,7 +347,8 @@
out.println("RecordType {");
Iterator<String> nameIter = r.getFieldNames().iterator();
Iterator<TypeExpression> typeIter = r.getFieldTypes().iterator();
- Iterator<Boolean> isOptionalIter = r.getOptionableFields().iterator();
+ Iterator<Boolean> isNullableIter = r.getNullableFields().iterator();
+ Iterator<Boolean> isMissableIter = r.getMissableFields().iterator();
boolean first = true;
while (nameIter.hasNext()) {
if (first) {
@@ -357,10 +358,11 @@
}
String name = nameIter.next();
TypeExpression texp = typeIter.next();
- Boolean isNullable = isOptionalIter.next();
+ Boolean isNullable = isNullableIter.next();
+ Boolean isMissable = isMissableIter.next();
out.print(skip(step + 1) + name + " : ");
texp.accept(this, step + 2);
- if (isNullable) {
+ if (isNullable || isMissable) {
out.print("?");
}
}
@@ -388,15 +390,18 @@
@Override
public Void visit(DatasetDecl dd, Integer step) throws CompilationException {
if (dd.getDatasetType() == DatasetType.INTERNAL) {
- String line = skip(step) + "DatasetDecl " + dd.getName() + "(" + dd.getItemTypeName() + ")"
- + " partitioned by " + ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getPartitioningExprs();
+ out.print(skip(step) + "DatasetDecl " + dd.getName() + "(");
+ dd.getItemType().accept(this, step + 2);
+ out.print(skip(step) + ") partitioned by "
+ + ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getPartitioningExprs());
if (((InternalDetailsDecl) dd.getDatasetDetailsDecl()).isAutogenerated()) {
- line += " [autogenerated]";
+ out.print(" [autogenerated]");
}
- out.println(line);
+ out.println();
} else if (dd.getDatasetType() == DatasetType.EXTERNAL) {
- out.println(skip(step) + "DatasetDecl " + dd.getName() + "(" + dd.getItemTypeName() + ")"
- + "is an external dataset");
+ out.print(skip(step) + "DatasetDecl " + dd.getName() + "(");
+ dd.getItemType().accept(this, step + 2);
+ out.println(skip(step) + ")is an external dataset");
}
return null;
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
index bde40de..db1485c 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
@@ -58,6 +58,9 @@
// Generate ids for variables (considering scopes) and replace global variable access with the dataset function.
variableCheckAndRewrite();
+ // Extracts SQL-92 aggregate functions from CASE/IF expressions into LET clauses
+ rewriteCaseExpressions();
+
// Rewrites SQL-92 global aggregations.
rewriteGroupByAggregationSugar();
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
index 8bc319f..8b3e12d 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
@@ -64,6 +64,7 @@
import org.apache.asterix.lang.sqlpp.rewrites.visitor.OperatorExpressionVisitor;
import org.apache.asterix.lang.sqlpp.rewrites.visitor.SetOperationVisitor;
import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlppBuiltinFunctionRewriteVisitor;
+import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlppCaseRewriteVisitor;
import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlppGroupByAggregationSugarVisitor;
import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlppGroupByVisitor;
import org.apache.asterix.lang.sqlpp.rewrites.visitor.SqlppInlineUdfsVisitor;
@@ -141,10 +142,13 @@
// Generate ids for variables (considering scopes) and replace global variable access with the dataset function.
variableCheckAndRewrite();
+ // Extracts SQL-92 aggregate functions from CASE/IF expressions into LET clauses
+ rewriteCaseExpressions();
+
// Rewrites SQL-92 aggregate functions
rewriteGroupByAggregationSugar();
- // Rewrite window expression aggregations.
+ // Rewrites window expression aggregations.
rewriteWindowAggregationSugar();
// Rewrites like/not-like expressions.
@@ -246,6 +250,11 @@
rewriteTopExpr(windowVisitor, null);
}
+ protected void rewriteCaseExpressions() throws CompilationException {
+ SqlppCaseRewriteVisitor visitor = new SqlppCaseRewriteVisitor(context);
+ rewriteTopExpr(visitor, null);
+ }
+
protected void inlineDeclaredUdfs(boolean inlineUdfs) throws CompilationException {
List<FunctionSignature> funIds = new ArrayList<FunctionSignature>();
for (FunctionDecl fdecl : declaredFunctions) {
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/AbstractSqlppExpressionExtractionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/AbstractSqlppExpressionExtractionVisitor.java
index 57bfad5..b132ea0 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/AbstractSqlppExpressionExtractionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/AbstractSqlppExpressionExtractionVisitor.java
@@ -23,23 +23,27 @@
import java.util.ArrayList;
import java.util.Deque;
import java.util.List;
+import java.util.function.Predicate;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.lang.common.base.AbstractClause;
import org.apache.asterix.lang.common.base.Expression;
import org.apache.asterix.lang.common.base.ILangExpression;
+import org.apache.asterix.lang.common.clause.GroupbyClause;
import org.apache.asterix.lang.common.clause.LetClause;
import org.apache.asterix.lang.common.expression.VariableExpr;
import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
import org.apache.asterix.lang.common.struct.VarIdentifier;
import org.apache.asterix.lang.sqlpp.clause.FromClause;
import org.apache.asterix.lang.sqlpp.clause.SelectBlock;
+import org.apache.asterix.lang.sqlpp.clause.SelectClause;
import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppSimpleExpressionVisitor;
import org.apache.hyracks.algebricks.common.utils.Pair;
/**
* Base class for visitors that extract expressions into LET clauses.
- * Subclasses should call {@link #extractExpressions(List, int)} to perform the extraction.
+ * Subclasses should call {@link #extractExpressionsFromList(List, int, Predicate)} or
+ * {@link #extractExpression(Expression)} to perform the extraction.
*/
abstract class AbstractSqlppExpressionExtractionVisitor extends AbstractSqlppSimpleExpressionVisitor {
@@ -57,32 +61,60 @@
stack.push(extractionList);
if (selectBlock.hasFromClause()) {
- FromClause clause = selectBlock.getFromClause();
- clause.accept(this, arg);
- if (!extractionList.isEmpty()) {
- handleUnsupportedClause(clause, extractionList);
- }
+ visitFromClause(selectBlock.getFromClause(), arg, extractionList);
}
List<AbstractClause> letWhereList = selectBlock.getLetWhereList();
if (!letWhereList.isEmpty()) {
- visitLetWhereClauses(letWhereList, extractionList, arg);
+ visitLetWhereClauses(letWhereList, arg, extractionList);
}
+ GroupbyClause groupbyClause = null;
if (selectBlock.hasGroupbyClause()) {
- selectBlock.getGroupbyClause().accept(this, arg);
- introduceLetClauses(extractionList, letWhereList);
+ groupbyClause = selectBlock.getGroupbyClause();
+ visitGroupByClause(groupbyClause, arg, extractionList, letWhereList);
}
List<AbstractClause> letHavingListAfterGby = selectBlock.getLetHavingListAfterGroupby();
if (!letHavingListAfterGby.isEmpty()) {
- visitLetWhereClauses(letHavingListAfterGby, extractionList, arg);
+ visitLetHavingClausesAfterGby(arg, extractionList, letHavingListAfterGby, groupbyClause);
}
- selectBlock.getSelectClause().accept(this, arg);
- introduceLetClauses(extractionList, selectBlock.hasGroupbyClause() ? letHavingListAfterGby : letWhereList);
+ visitSelectClause(selectBlock.getSelectClause(), arg, extractionList,
+ selectBlock.hasGroupbyClause() ? letHavingListAfterGby : letWhereList, groupbyClause);
stack.pop();
return null;
}
- private void visitLetWhereClauses(List<AbstractClause> clauseList,
+ protected void visitFromClause(FromClause clause, ILangExpression arg,
+ List<Pair<Expression, VarIdentifier>> extractionList) throws CompilationException {
+ // Skip extraction because we won't be able to perform it as there are no LET clauses yet.
+ // Subclasses can override and fail if necessary
+ }
+
+ protected void visitLetWhereClauses(List<AbstractClause> letWhereList, ILangExpression arg,
+ List<Pair<Expression, VarIdentifier>> extractionList) throws CompilationException {
+ visitLetWhereClausesImpl(letWhereList, extractionList, arg);
+ }
+
+ protected void visitGroupByClause(GroupbyClause groupbyClause, ILangExpression arg,
+ List<Pair<Expression, VarIdentifier>> extractionList, List<AbstractClause> letWhereList)
+ throws CompilationException {
+ groupbyClause.accept(this, arg);
+ introduceLetClauses(extractionList, letWhereList);
+ }
+
+ protected void visitLetHavingClausesAfterGby(ILangExpression arg,
+ List<Pair<Expression, VarIdentifier>> extractionList, List<AbstractClause> letHavingListAfterGby,
+ GroupbyClause groupbyClause) throws CompilationException {
+ visitLetWhereClausesImpl(letHavingListAfterGby, extractionList, arg);
+ }
+
+ protected void visitSelectClause(SelectClause selectClause, ILangExpression arg,
+ List<Pair<Expression, VarIdentifier>> extractionList, List<AbstractClause> letWhereList,
+ GroupbyClause groupbyClause) throws CompilationException {
+ selectClause.accept(this, arg);
+ introduceLetClauses(extractionList, letWhereList);
+ }
+
+ private void visitLetWhereClausesImpl(List<AbstractClause> clauseList,
List<Pair<Expression, VarIdentifier>> extractionList, ILangExpression arg) throws CompilationException {
List<AbstractClause> newClauseList = new ArrayList<>(clauseList.size());
for (AbstractClause letWhereClause : clauseList) {
@@ -108,7 +140,8 @@
fromBindingList.clear();
}
- List<Expression> extractExpressions(List<Expression> exprList, int limit) {
+ protected List<Expression> extractExpressionsFromList(List<Expression> exprList, int limit,
+ Predicate<Expression> exprTest) {
List<Pair<Expression, VarIdentifier>> outLetList = stack.peek();
if (outLetList == null) {
return null;
@@ -117,23 +150,22 @@
List<Expression> newExprList = new ArrayList<>(n);
for (int i = 0; i < n; i++) {
Expression expr = exprList.get(i);
- Expression newExpr;
- if (i < limit && isExtractableExpression(expr)) {
- VarIdentifier v = context.newVariable();
- VariableExpr vExpr = new VariableExpr(v);
- vExpr.setSourceLocation(expr.getSourceLocation());
- outLetList.add(new Pair<>(expr, v));
- newExpr = vExpr;
- } else {
- newExpr = expr;
- }
+ Expression newExpr = i < limit && exprTest.test(expr) ? extractExpressionImpl(expr, outLetList) : expr;
newExprList.add(newExpr);
}
return newExprList;
}
- abstract boolean isExtractableExpression(Expression expr);
+ protected Expression extractExpression(Expression expr) {
+ List<Pair<Expression, VarIdentifier>> outLetList = stack.peek();
+ return outLetList != null ? extractExpressionImpl(expr, outLetList) : null;
+ }
- abstract void handleUnsupportedClause(FromClause clause, List<Pair<Expression, VarIdentifier>> extractionList)
- throws CompilationException;
+ private VariableExpr extractExpressionImpl(Expression expr, List<Pair<Expression, VarIdentifier>> outLetList) {
+ VarIdentifier v = context.newVariable();
+ VariableExpr vExpr = new VariableExpr(v);
+ vExpr.setSourceLocation(expr.getSourceLocation());
+ outLetList.add(new Pair<>(expr, v));
+ return vExpr;
+ }
}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
index 23b4d60..9c49071 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/OperatorExpressionVisitor.java
@@ -103,6 +103,7 @@
comparison.addOperand(itemExpr);
comparison.addOperand(bindingVar);
comparison.setCurrentop(true);
+ comparison.addHints(operatorExpr.getHints());
comparison.setSourceLocation(operatorExpr.getSourceLocation());
if (opType == OperatorType.IN) {
comparison.addOperator(OperatorType.EQ);
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppCaseRewriteVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppCaseRewriteVisitor.java
new file mode 100644
index 0000000..5752aef
--- /dev/null
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppCaseRewriteVisitor.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.lang.sqlpp.rewrites.visitor;
+
+import java.util.List;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.lang.common.base.AbstractClause;
+import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.ILangExpression;
+import org.apache.asterix.lang.common.clause.GroupbyClause;
+import org.apache.asterix.lang.common.expression.CallExpr;
+import org.apache.asterix.lang.common.expression.IfExpr;
+import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
+import org.apache.asterix.lang.common.struct.VarIdentifier;
+import org.apache.asterix.lang.sqlpp.expression.CaseExpression;
+import org.apache.asterix.lang.sqlpp.expression.SelectExpression;
+import org.apache.asterix.lang.sqlpp.util.FunctionMapUtil;
+import org.apache.asterix.lang.sqlpp.visitor.base.AbstractSqlppSimpleExpressionVisitor;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+
+/**
+ * Extracts SQL-92 aggregate functions from CASE/IF expressions into LET clauses,
+ * so they can be pushed into GROUPBY subplans by the optimizer.
+ */
+public final class SqlppCaseRewriteVisitor extends AbstractSqlppExpressionExtractionVisitor {
+
+ public SqlppCaseRewriteVisitor(LangRewritingContext context) {
+ super(context);
+ }
+
+ @Override
+ protected void visitLetWhereClauses(List<AbstractClause> letWhereList, ILangExpression arg,
+ List<Pair<Expression, VarIdentifier>> extractionList) {
+ // do not perform the extraction
+ }
+
+ @Override
+ protected void visitGroupByClause(GroupbyClause groupbyClause, ILangExpression arg,
+ List<Pair<Expression, VarIdentifier>> extractionList, List<AbstractClause> letWhereList) {
+ // do not perform the extraction
+ }
+
+ @Override
+ public Expression visit(CaseExpression caseExpr, ILangExpression arg) throws CompilationException {
+ Expression resultExpr = super.visit(caseExpr, arg);
+ resultExpr.accept(new Sql92AggregateExtractionVisitor(), arg);
+ return resultExpr;
+ }
+
+ @Override
+ public Expression visit(IfExpr ifExpr, ILangExpression arg) throws CompilationException {
+ Expression resultExpr = super.visit(ifExpr, arg);
+ resultExpr.accept(new Sql92AggregateExtractionVisitor(), arg);
+ return resultExpr;
+ }
+
+ private final class Sql92AggregateExtractionVisitor extends AbstractSqlppSimpleExpressionVisitor {
+
+ @Override
+ public Expression visit(CallExpr callExpr, ILangExpression arg) throws CompilationException {
+ CallExpr resultExpr = (CallExpr) super.visit(callExpr, arg);
+ if (FunctionMapUtil.isSql92AggregateFunction(resultExpr.getFunctionSignature())) {
+ Expression newExpr = extractExpression(resultExpr);
+ if (newExpr != null) {
+ return newExpr;
+ }
+ }
+ return resultExpr;
+ }
+
+ @Override
+ public Expression visit(SelectExpression selectExpression, ILangExpression arg) {
+ // don't visit sub-queries
+ return selectExpression;
+ }
+ }
+}
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java
index a5f43b6..bed2fed 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowAggregationSugarVisitor.java
@@ -105,8 +105,11 @@
Map<VariableExpr, Set<? extends Scope.SymbolAnnotation>> liveAnnotatedVars =
scopeChecker.getCurrentScope().getLiveVariables();
Set<VariableExpr> liveVars = liveAnnotatedVars.keySet();
- Set<VariableExpr> liveContextVars = Scope.findVariablesAnnotatedBy(liveVars,
- SqlppVariableAnnotation.CONTEXT_VARIABLE, liveAnnotatedVars, winExpr.getSourceLocation());
+
+ Map<VariableExpr, Set<? extends Scope.SymbolAnnotation>> localAnnotatedVars =
+ scopeChecker.getCurrentScope().getLiveVariables(scopeChecker.getPrecedingScope());
+ Set<VariableExpr> liveContextVars = Scope.findVariablesAnnotatedBy(localAnnotatedVars.keySet(),
+ SqlppVariableAnnotation.CONTEXT_VARIABLE, localAnnotatedVars, winExpr.getSourceLocation());
List<Pair<Expression, Identifier>> winFieldList = winExpr.getWindowFieldList();
Map<VariableExpr, Identifier> winVarFieldMap =
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowRewriteVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowRewriteVisitor.java
index ba1e7f9..9167ce8 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowRewriteVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/visitor/SqlppWindowRewriteVisitor.java
@@ -53,6 +53,16 @@
}
@Override
+ protected void visitFromClause(FromClause clause, ILangExpression arg,
+ List<Pair<Expression, VarIdentifier>> extractionList) throws CompilationException {
+ clause.accept(this, arg);
+ if (!extractionList.isEmpty()) {
+ throw new CompilationException(ErrorCode.COMPILATION_UNEXPECTED_WINDOW_EXPRESSION,
+ clause.getSourceLocation());
+ }
+ }
+
+ @Override
public Expression visit(WindowExpression winExpr, ILangExpression arg) throws CompilationException {
super.visit(winExpr, arg);
@@ -68,14 +78,16 @@
rewriteSpecificWindowFunctions(winfi, winExpr);
if (BuiltinFunctions.builtinFunctionHasProperty(winfi,
BuiltinFunctions.WindowFunctionProperty.HAS_LIST_ARG)) {
- List<Expression> newExprList = extractExpressions(winExpr.getExprList(), 1);
+ List<Expression> newExprList = extractExpressionsFromList(winExpr.getExprList(), 1,
+ SqlppWindowRewriteVisitor::isExtractableExpression);
if (newExprList == null) {
throw new CompilationException(ErrorCode.COMPILATION_ERROR, winExpr.getSourceLocation(), "");
}
winExpr.setExprList(newExprList);
}
} else if (FunctionMapUtil.isSql92AggregateFunction(signature)) {
- List<Expression> newExprList = extractExpressions(winExpr.getExprList(), winExpr.getExprList().size());
+ List<Expression> newExprList = extractExpressionsFromList(winExpr.getExprList(),
+ winExpr.getExprList().size(), SqlppWindowRewriteVisitor::isExtractableExpression);
if (newExprList == null) {
throw new CompilationException(ErrorCode.COMPILATION_ERROR, winExpr.getSourceLocation(), "");
}
@@ -88,8 +100,7 @@
return winExpr;
}
- @Override
- protected boolean isExtractableExpression(Expression expr) {
+ protected static boolean isExtractableExpression(Expression expr) {
switch (expr.getKind()) {
case LITERAL_EXPRESSION:
case VARIABLE_EXPRESSION:
@@ -99,12 +110,6 @@
}
}
- @Override
- void handleUnsupportedClause(FromClause clause, List<Pair<Expression, VarIdentifier>> extractionList)
- throws CompilationException {
- throw new CompilationException(ErrorCode.COMPILATION_UNEXPECTED_WINDOW_EXPRESSION, clause.getSourceLocation());
- }
-
/**
* Apply rewritings for specific window functions:
* <ul>
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index e0a4341..4c924a4 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -335,15 +335,15 @@
hintCollector.clear();
try {
return parseFunction.parse();
+ } catch (SqlppParseException e) {
+ throw new CompilationException(ErrorCode.PARSE_ERROR, e.getSourceLocation(), LogRedactionUtil.userData(getMessage(e)));
+ } catch (ParseException e) {
+ throw new CompilationException(ErrorCode.PARSE_ERROR, LogRedactionUtil.userData(getMessage(e)));
} catch (Error e) {
// this is here as the JavaCharStream that's below the lexer sometimes throws Errors that are not handled
// by the ANTLR-generated lexer or parser (e.g it does this for invalid backslash u + 4 hex digits escapes)
final String msg = e.getClass().getSimpleName() + (e.getMessage() != null ? ": " + e.getMessage() : "");
throw new CompilationException(ErrorCode.PARSE_ERROR, LogRedactionUtil.userData(msg));
- } catch (SqlppParseException e) {
- throw new CompilationException(ErrorCode.PARSE_ERROR, e.getSourceLocation(), LogRedactionUtil.userData(getMessage(e)));
- } catch (ParseException e) {
- throw new CompilationException(ErrorCode.PARSE_ERROR, LogRedactionUtil.userData(getMessage(e)));
} finally {
reportUnclaimedHints();
}
@@ -428,7 +428,11 @@
}
private SqlppParseException createUnexpectedTokenError() {
- return new SqlppParseException(getSourceLocation(token), "Unexpected token: " + LogRedactionUtil.userData(token.image));
+ return createUnexpectedTokenError(token);
+ }
+
+ private SqlppParseException createUnexpectedTokenError(Token t) {
+ return new SqlppParseException(getSourceLocation(t), "Unexpected token: " + LogRedactionUtil.userData(t.image));
}
private boolean laToken(int idx, int kind, String image) {
@@ -646,7 +650,8 @@
{
Pair<Identifier,Identifier> nameComponents = null;
boolean ifNotExists = false;
- Pair<Identifier,Identifier> typeComponents = null;
+ TypeExpression typeExpr = null;
+ TypeExpression metaTypeExpr = null;
String adapterName = null;
Map<String,String> properties = null;
FunctionSignature appliedFunction = null;
@@ -656,13 +661,12 @@
DatasetDecl stmt = null;
boolean autogenerated = false;
Pair<Integer, List<String>> filterField = null;
- Pair<Identifier,Identifier> metaTypeComponents = new Pair<Identifier, Identifier>(null, null);
RecordConstructor withRecord = null;
}
{
(
<EXTERNAL> Dataset() nameComponents = QualifiedName()
- <LEFTPAREN> typeComponents = TypeName() <RIGHTPAREN>
+ typeExpr = DatasetTypeSpecification()
ifNotExists = IfNotExists()
<USING> adapterName = AdapterName() properties = Configuration()
( <ON> nodeGroupName = Identifier() )?
@@ -675,10 +679,8 @@
try{
stmt = new DatasetDecl(nameComponents.first,
nameComponents.second,
- typeComponents.first,
- typeComponents.second,
- metaTypeComponents.first,
- metaTypeComponents.second,
+ typeExpr,
+ null,
nodeGroupName != null? new Identifier(nodeGroupName): null,
hints,
DatasetType.EXTERNAL,
@@ -692,7 +694,7 @@
| ( <INTERNAL> )?
Dataset() nameComponents = QualifiedName()
- <LEFTPAREN> typeComponents = TypeName() <RIGHTPAREN>
+ typeExpr = DatasetTypeSpecification()
(
{ String name; }
<WITH>
@@ -703,7 +705,7 @@
"We can only support one additional associated field called \"meta\".");
}
}
- <LEFTPAREN> metaTypeComponents = TypeName() <RIGHTPAREN>
+ metaTypeExpr = DatasetTypeSpecification()
)?
ifNotExists = IfNotExists()
primaryKeyFields = PrimaryKey()
@@ -724,10 +726,8 @@
try{
stmt = new DatasetDecl(nameComponents.first,
nameComponents.second,
- typeComponents.first,
- typeComponents.second,
- metaTypeComponents.first,
- metaTypeComponents.second,
+ typeExpr,
+ metaTypeExpr,
nodeGroupName != null ? new Identifier(nodeGroupName) : null,
hints,
DatasetType.INTERNAL,
@@ -744,6 +744,76 @@
}
}
+TypeExpression DatasetTypeSpecification() throws ParseException:
+{
+ TypeExpression typeExpr = null;
+}
+{
+ (
+ LOOKAHEAD(3) typeExpr = DatasetRecordTypeSpecification(true)
+ | typeExpr = DatasetReferenceTypeSpecification()
+ )
+ {
+ return typeExpr;
+ }
+}
+
+TypeExpression DatasetReferenceTypeSpecification() throws ParseException:
+{
+ TypeExpression typeExpr = null;
+}
+{
+ <LEFTPAREN> typeExpr = TypeReference() <RIGHTPAREN>
+ {
+ return typeExpr;
+ }
+}
+
+TypeExpression DatasetRecordTypeSpecification(boolean allowRecordKindModifier) throws ParseException:
+{
+ RecordTypeDefinition recordTypeDef = null;
+ RecordTypeDefinition.RecordKind recordKind = null;
+ Token startToken = null, recordKindToken = null;
+}
+{
+ <LEFTPAREN> { startToken = token; } recordTypeDef = DatasetRecordTypeDef() <RIGHTPAREN>
+ ( recordKind = RecordTypeKind() { recordKindToken = token; } <TYPE> )?
+ {
+ if (recordKind == null) {
+ recordKind = RecordTypeDefinition.RecordKind.CLOSED;
+ } else if (!allowRecordKindModifier) {
+ throw createUnexpectedTokenError(recordKindToken);
+ }
+ recordTypeDef.setRecordKind(recordKind);
+ return addSourceLocation(recordTypeDef, startToken);
+ }
+}
+
+RecordTypeDefinition DatasetRecordTypeDef() throws ParseException:
+{
+ RecordTypeDefinition recType = new RecordTypeDefinition();
+}
+{
+ DatasetRecordField(recType) ( <COMMA> DatasetRecordField(recType) )*
+ {
+ return recType;
+ }
+}
+
+void DatasetRecordField(RecordTypeDefinition recType) throws ParseException:
+{
+ String fieldName;
+ TypeExpression type = null;
+ boolean nullable = true, missable = true;
+}
+{
+ fieldName = Identifier()
+ type = TypeReference() ( <NOT> <UNKNOWN> { nullable = false; missable = false; } )?
+ {
+ recType.addField(fieldName, type, nullable, missable);
+ }
+}
+
RefreshExternalDatasetStatement RefreshExternalDatasetStatement() throws ParseException:
{
Token startToken = null;
@@ -1494,7 +1564,9 @@
String value;
}
{
- <LEFTPAREN> key = ConstantString() <EQ> value = ConstantString() <RIGHTPAREN>
+ <LEFTPAREN> key = ConstantString()
+ <EQ> ( value = ConstantString() | (<TRUE> | <FALSE>) {value = token.image.toLowerCase();} )
+ <RIGHTPAREN>
{
return new Pair<String, String>(key, value);
}
@@ -1575,15 +1647,28 @@
}
}
+RecordTypeDefinition.RecordKind RecordTypeKind() throws ParseException:
+{
+ RecordTypeDefinition.RecordKind recordKind = null;
+}
+{
+ (
+ <CLOSED> { recordKind = RecordTypeDefinition.RecordKind.CLOSED; }
+ | <OPEN> { recordKind = RecordTypeDefinition.RecordKind.OPEN; }
+ )
+ {
+ return recordKind;
+ }
+}
+
RecordTypeDefinition RecordTypeDef() throws ParseException:
{
Token startToken = null;
RecordTypeDefinition recType = new RecordTypeDefinition();
- RecordTypeDefinition.RecordKind recordKind = null;
+ RecordTypeDefinition.RecordKind recordKind = RecordTypeDefinition.RecordKind.OPEN;
}
{
- ( <CLOSED> { recordKind = RecordTypeDefinition.RecordKind.CLOSED; }
- | <OPEN> { recordKind = RecordTypeDefinition.RecordKind.OPEN; } )?
+ ( recordKind = RecordTypeKind() )?
<LEFTBRACE>
{
startToken = token;
@@ -1610,9 +1695,6 @@
)?
<RIGHTBRACE>
{
- if (recordKind == null) {
- recordKind = RecordTypeDefinition.RecordKind.OPEN;
- }
recType.setRecordKind(recordKind);
return addSourceLocation(recType, startToken);
}
@@ -1622,7 +1704,7 @@
{
String fieldName;
TypeExpression type = null;
- boolean nullable = false;
+ boolean nullable = false, missable = false;
}
{
fieldName = Identifier()
@@ -1632,9 +1714,9 @@
SqlppHint.DATE_BETWEEN_YEARS_HINT, SqlppHint.DATETIME_ADD_RAND_HOURS_HINT, SqlppHint.AUTO_HINT);
IRecordFieldDataGen rfdg = hintToken != null ? parseFieldDataGen(hintToken) : null;
}
- <COLON> type = TypeExpr() (<QUES> { nullable = true; } )?
+ <COLON> type = TypeExpr() ( <QUES> { nullable = true; missable = true; } )?
{
- recType.addField(fieldName, type, nullable, rfdg);
+ recType.addField(fieldName, type, nullable, missable, rfdg);
}
}
@@ -3417,12 +3499,12 @@
UnnestClause UnnestClause(JoinType joinType) throws ParseException :
{
Token startToken = null;
- Expression rightExpr;
- VariableExpr rightVar;
+ Expression rightExpr = null;
+ VariableExpr rightVar = null;
VariableExpr posVar = null;
}
{
- (<UNNEST>|<CORRELATE>|<FLATTEN>) { startToken = token; } rightExpr = Expression() ((<AS>)? rightVar = Variable()) (<AT> posVar = Variable())?
+ (<UNNEST>|<CORRELATE>|<FLATTEN>) { startToken = token; } rightExpr = Expression() ((<AS>)? rightVar = Variable())? (<AT> posVar = Variable())?
{
if (rightVar == null) {
rightVar = ExpressionToVariableUtil.getGeneratedVariable(rightExpr, true);
diff --git a/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/pom.xml b/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/pom.xml
index fb78a92..f4b811b 100644
--- a/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/pom.xml
+++ b/asterixdb/asterix-maven-plugins/asterix-grammar-extension-maven-plugin/pom.xml
@@ -45,7 +45,6 @@
<dependency>
<groupId>org.apache.maven.plugin-tools</groupId>
<artifactId>maven-plugin-annotations</artifactId>
- <version>3.6.0</version>
<scope>provided</scope>
</dependency>
<dependency>
diff --git a/asterixdb/asterix-metadata/pom.xml b/asterixdb/asterix-metadata/pom.xml
index d14d8d5..a5425b1 100644
--- a/asterixdb/asterix-metadata/pom.xml
+++ b/asterixdb/asterix-metadata/pom.xml
@@ -155,7 +155,7 @@
</dependency>
<dependency>
<groupId>org.mockito</groupId>
- <artifactId>mockito-all</artifactId>
+ <artifactId>mockito-core</artifactId>
<scope>test</scope>
</dependency>
<dependency>
@@ -178,5 +178,9 @@
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdfs-client</artifactId>
</dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-ipc</artifactId>
+ </dependency>
</dependencies>
</project>
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
index e5ee163..362ead8 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataManager.java
@@ -129,6 +129,11 @@
@Override
public MetadataTransactionContext beginTransaction() throws RemoteException {
+ try {
+ INSTANCE.init();
+ } catch (HyracksDataException e) {
+ throw new ACIDException(e);
+ }
TxnId txnId = createTxnId();
metadataNode.beginTransaction(txnId);
return new MetadataTransactionContext(txnId);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
index e6a768d..5d4fe1c 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
@@ -328,7 +328,7 @@
ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(dataverse);
insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.DATAVERSE_DATASET, tuple);
} catch (HyracksDataException e) {
- if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
+ if (e.matches(ErrorCode.DUPLICATE_KEY)) {
throw new AlgebricksException(
"A dataverse with this name " + dataverse.getDataverseName() + " already exists.", e);
} else {
@@ -354,7 +354,7 @@
addIndex(txnId, primaryIndex);
}
} catch (HyracksDataException e) {
- if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
+ if (e.matches(ErrorCode.DUPLICATE_KEY)) {
throw new AlgebricksException("A dataset with this name " + dataset.getDatasetName()
+ " already exists in dataverse '" + dataset.getDataverseName() + "'.", e);
} else {
@@ -370,7 +370,7 @@
ITupleReference tuple = tupleWriter.getTupleFromMetadataEntity(index);
insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
} catch (HyracksDataException e) {
- if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
+ if (e.matches(ErrorCode.DUPLICATE_KEY)) {
throw new AlgebricksException("An index with name '" + index.getIndexName() + "' already exists.", e);
} else {
throw new AlgebricksException(e);
@@ -385,7 +385,7 @@
ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(node);
insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.NODE_DATASET, tuple);
} catch (HyracksDataException e) {
- if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
+ if (e.matches(ErrorCode.DUPLICATE_KEY)) {
throw new AlgebricksException("A node with name '" + node.getNodeName() + "' already exists.", e);
} else {
throw new AlgebricksException(e);
@@ -401,7 +401,7 @@
ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(nodeGroup);
modifyMetadataIndex(modificationOp, txnId, MetadataPrimaryIndexes.NODEGROUP_DATASET, tuple);
} catch (HyracksDataException e) {
- if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
+ if (e.matches(ErrorCode.DUPLICATE_KEY)) {
throw new AlgebricksException(
"A nodegroup with name '" + nodeGroup.getNodeGroupName() + "' already exists.", e);
} else {
@@ -418,7 +418,7 @@
ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(datatype);
insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
} catch (HyracksDataException e) {
- if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
+ if (e.matches(ErrorCode.DUPLICATE_KEY)) {
throw new AlgebricksException(
"A datatype with name '" + datatype.getDatatypeName() + "' already exists.", e);
} else {
@@ -437,7 +437,7 @@
insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.FUNCTION_DATASET, functionTuple);
} catch (HyracksDataException e) {
- if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
+ if (e.matches(ErrorCode.DUPLICATE_KEY)) {
throw new AlgebricksException("A function with this name " + function.getName() + " and arity "
+ function.getArity() + " already exists in dataverse '" + function.getDataverseName() + "'.",
e);
@@ -583,8 +583,7 @@
ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.DATAVERSE_DATASET, searchKey);
deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATAVERSE_DATASET, tuple);
} catch (HyracksDataException e) {
- if (e.getComponent().equals(ErrorCode.HYRACKS)
- && e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
+ if (e.matches(ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)) {
throw new AlgebricksException("Cannot drop dataverse '" + dataverseName + "' because it doesn't exist.",
e);
} else {
@@ -642,8 +641,7 @@
} catch (HyracksDataException hde) {
// ignore this exception and continue deleting all relevant
// artifacts.
- if (!hde.getComponent().equals(ErrorCode.HYRACKS)
- || hde.getErrorCode() != ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
+ if (!hde.matches(ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)) {
throw new AlgebricksException(hde);
}
} finally {
@@ -664,8 +662,7 @@
ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey);
deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
} catch (HyracksDataException e) {
- if (e.getComponent().equals(ErrorCode.HYRACKS)
- && e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
+ if (e.matches(ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)) {
throw new AlgebricksException(
"Cannot drop index '" + datasetName + "." + indexName + "' because it doesn't exist.", e);
} else {
@@ -698,8 +695,7 @@
deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.NODEGROUP_DATASET, tuple);
return true;
} catch (HyracksDataException e) {
- if (e.getComponent().equals(ErrorCode.HYRACKS)
- && e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
+ if (e.matches(ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)) {
throw new AlgebricksException("Cannot drop nodegroup '" + nodeGroupName + "' because it doesn't exist",
e);
} else {
@@ -730,8 +726,7 @@
}
}
} catch (HyracksDataException e) {
- if (e.getComponent().equals(ErrorCode.HYRACKS)
- && e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
+ if (e.matches(ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)) {
throw new AlgebricksException("Cannot drop type '" + datatypeName + "' because it doesn't exist", e);
} else {
throw new AlgebricksException(e);
@@ -748,8 +743,7 @@
ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey);
deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
} catch (HyracksDataException e) {
- if (e.getComponent().equals(ErrorCode.HYRACKS)
- && e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
+ if (e.matches(ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)) {
throw new AlgebricksException("Cannot drop type '" + datatypeName + "' because it doesn't exist", e);
} else {
throw new AlgebricksException(e);
@@ -1191,8 +1185,7 @@
getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey);
deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.FUNCTION_DATASET, functionTuple);
} catch (HyracksDataException e) {
- if (e.getComponent().equals(ErrorCode.HYRACKS)
- && e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
+ if (e.matches(ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)) {
throw new AlgebricksException("There is no function with the name " + functionSignature.getName()
+ " and arity " + functionSignature.getArity(), e);
} else {
@@ -1433,7 +1426,7 @@
ITupleReference adapterTuple = tupleReaderWriter.getTupleFromMetadataEntity(adapter);
insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, adapterTuple);
} catch (HyracksDataException e) {
- if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
+ if (e.matches(ErrorCode.DUPLICATE_KEY)) {
throw new AlgebricksException("A adapter with this name " + adapter.getAdapterIdentifier().getName()
+ " already exists in dataverse '" + adapter.getAdapterIdentifier().getNamespace() + "'.", e);
} else {
@@ -1458,8 +1451,7 @@
getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey);
deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, datasetTuple);
} catch (HyracksDataException e) {
- if (e.getComponent().equals(ErrorCode.HYRACKS)
- && e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
+ if (e.matches(ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)) {
throw new AlgebricksException("Cannot drop adapter '" + adapterName + " since it doesn't exist", e);
} else {
throw new AlgebricksException(e);
@@ -1496,8 +1488,8 @@
ITupleReference compactionPolicyTuple = tupleReaderWriter.getTupleFromMetadataEntity(compactionPolicy);
insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.COMPACTION_POLICY_DATASET, compactionPolicyTuple);
} catch (HyracksDataException e) {
- if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
- throw new AlgebricksException("A compcation policy with this name " + compactionPolicy.getPolicyName()
+ if (e.matches(ErrorCode.DUPLICATE_KEY)) {
+ throw new AlgebricksException("A compaction policy with this name " + compactionPolicy.getPolicyName()
+ " already exists in dataverse '" + compactionPolicy.getPolicyName() + "'.", e);
} else {
throw new AlgebricksException(e);
@@ -1549,7 +1541,7 @@
insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.LIBRARY_DATASET, libraryTuple);
} catch (HyracksDataException e) {
- if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
+ if (e.matches(ErrorCode.DUPLICATE_KEY)) {
throw new AlgebricksException("A library with this name " + library.getDataverseName()
+ " already exists in dataverse '" + library.getDataverseName() + "'.", e);
} else {
@@ -1574,8 +1566,7 @@
getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.LIBRARY_DATASET, searchKey);
deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.LIBRARY_DATASET, datasetTuple);
} catch (HyracksDataException e) {
- if (e.getComponent().equals(ErrorCode.HYRACKS)
- && e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
+ if (e.matches(ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)) {
throw new AlgebricksException("Cannot drop library '" + libraryName, e);
} else {
throw new AlgebricksException(e);
@@ -1614,7 +1605,7 @@
ITupleReference feedPolicyTuple = tupleReaderWriter.getTupleFromMetadataEntity(feedPolicy);
insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.FEED_POLICY_DATASET, feedPolicyTuple);
} catch (HyracksDataException e) {
- if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
+ if (e.matches(ErrorCode.DUPLICATE_KEY)) {
throw new AlgebricksException("A feed policy with this name " + feedPolicy.getPolicyName()
+ " already exists in dataverse '" + feedPolicy.getPolicyName() + "'.", e);
} else {
@@ -1706,7 +1697,7 @@
ITupleReference feedTuple = tupleReaderWriter.getTupleFromMetadataEntity(feed);
insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.FEED_DATASET, feedTuple);
} catch (HyracksDataException e) {
- if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
+ if (e.matches(ErrorCode.DUPLICATE_KEY)) {
throw new AlgebricksException("A feed with this name " + feed.getFeedName()
+ " already exists in dataverse '" + feed.getDataverseName() + "'.", e);
} else {
@@ -1755,8 +1746,7 @@
ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.FEED_DATASET, searchKey);
deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.FEED_DATASET, tuple);
} catch (HyracksDataException e) {
- if (e.getComponent().equals(ErrorCode.HYRACKS)
- && e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
+ if (e.matches(ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)) {
throw new AlgebricksException("Cannot drop feed '" + feedName + "' because it doesn't exist", e);
} else {
throw new AlgebricksException(e);
@@ -1772,8 +1762,7 @@
ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.FEED_POLICY_DATASET, searchKey);
deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.FEED_POLICY_DATASET, tuple);
} catch (HyracksDataException e) {
- if (e.getComponent().equals(ErrorCode.HYRACKS)
- && e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
+ if (e.matches(ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)) {
throw new AlgebricksException("Unknown feed policy " + policyName, e);
} else {
throw new AlgebricksException(e);
@@ -1805,7 +1794,7 @@
ITupleReference externalFileTuple = tupleReaderWriter.getTupleFromMetadataEntity(externalFile);
insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, externalFileTuple);
} catch (HyracksDataException e) {
- if (e.getComponent().equals(ErrorCode.HYRACKS) && e.getErrorCode() == ErrorCode.DUPLICATE_KEY) {
+ if (e.matches(ErrorCode.DUPLICATE_KEY)) {
throw new AlgebricksException("An externalFile with this number " + externalFile.getFileNumber()
+ " already exists in dataset '" + externalFile.getDatasetName() + "' in dataverse '"
+ externalFile.getDataverseName() + "'.", e);
@@ -1843,8 +1832,7 @@
getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, searchKey);
deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, datasetTuple);
} catch (HyracksDataException e) {
- if (e.getComponent().equals(ErrorCode.HYRACKS)
- && e.getErrorCode() == ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
+ if (e.matches(ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)) {
throw new AlgebricksException("Couldn't drop externalFile.", e);
} else {
throw new AlgebricksException(e);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/RMIClientFactory.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/RMIClientFactory.java
index ac2ecd0..515e763 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/RMIClientFactory.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/RMIClientFactory.java
@@ -21,23 +21,72 @@
import java.io.IOException;
import java.io.Serializable;
+import java.net.InetAddress;
import java.net.Socket;
import java.rmi.server.RMIClientSocketFactory;
import javax.net.SocketFactory;
import javax.net.ssl.SSLSocketFactory;
+import org.apache.hyracks.api.network.INetworkSecurityConfig;
+import org.apache.hyracks.ipc.security.NetworkSecurityManager;
+
public class RMIClientFactory implements RMIClientSocketFactory, Serializable {
+
private static final long serialVersionUID = -3874278041718817394L;
+ private final INetworkSecurityConfig config;
+ private transient SocketFactory socketFactory;
- private final boolean sslEnabled;
+ public RMIClientFactory(INetworkSecurityConfig config) {
+ this.config = config;
- public RMIClientFactory(boolean sslEnabled) {
- this.sslEnabled = sslEnabled;
}
public Socket createSocket(String host, int port) throws IOException {
- final SocketFactory factory = sslEnabled ? SSLSocketFactory.getDefault() : SocketFactory.getDefault();
- return factory.createSocket(host, port);
+ synchronized (this) {
+ if (socketFactory == null) {
+ socketFactory = config.isSslEnabled() ? new RMITrustedClientSSLSocketFactory(config)
+ : SocketFactory.getDefault();
+ }
+ }
+ return socketFactory.createSocket(host, port);
+ }
+
+ private static class RMITrustedClientSSLSocketFactory extends SSLSocketFactory {
+
+ protected SSLSocketFactory factory;
+
+ public RMITrustedClientSSLSocketFactory(INetworkSecurityConfig config) {
+ this.factory = NetworkSecurityManager.newSSLContext(config).getSocketFactory();
+ }
+
+ public Socket createSocket(InetAddress host, int port) throws IOException {
+ return this.factory.createSocket(host, port);
+ }
+
+ public Socket createSocket(String host, int port) throws IOException {
+ return this.factory.createSocket(host, port);
+ }
+
+ public Socket createSocket(String host, int port, InetAddress localHost, int localPort) throws IOException {
+ return this.factory.createSocket(host, port, localHost, localPort);
+ }
+
+ public Socket createSocket(InetAddress address, int port, InetAddress localAddress, int localPort)
+ throws IOException {
+ return this.factory.createSocket(address, port, localAddress, localPort);
+ }
+
+ public Socket createSocket(Socket socket, String host, int port, boolean autoClose) throws IOException {
+ return this.factory.createSocket(socket, host, port, autoClose);
+ }
+
+ public String[] getDefaultCipherSuites() {
+ return this.factory.getDefaultCipherSuites();
+ }
+
+ public String[] getSupportedCipherSuites() {
+ return this.factory.getSupportedCipherSuites();
+ }
}
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/AsterixStateProxy.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/AsterixStateProxy.java
index d6af749..2104fdf 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/AsterixStateProxy.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/AsterixStateProxy.java
@@ -49,7 +49,7 @@
if (networkSecurityManager.getConfiguration().isSslEnabled()) {
final RMIServerFactory serverSocketFactory = new RMIServerFactory(networkSecurityManager);
final RMIClientFactory clientSocketFactory =
- new RMIClientFactory(networkSecurityManager.getConfiguration().isSslEnabled());
+ new RMIClientFactory(networkSecurityManager.getConfiguration());
stub = (IAsterixStateProxy) UnicastRemoteObject.exportObject(cc, metadataCallbackPort, clientSocketFactory,
serverSocketFactory);
} else {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
index 49fffe6..3412941 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -35,13 +35,13 @@
import org.apache.asterix.common.context.IStorageComponentProvider;
import org.apache.asterix.common.exceptions.ACIDException;
import org.apache.asterix.common.exceptions.MetadataException;
+import org.apache.asterix.common.external.IDataSourceAdapter;
import org.apache.asterix.common.ioopcallbacks.LSMIndexIOOperationCallbackFactory;
import org.apache.asterix.common.ioopcallbacks.LSMIndexPageWriteCallbackFactory;
import org.apache.asterix.common.utils.StorageConstants;
import org.apache.asterix.common.utils.StoragePathUtil;
import org.apache.asterix.external.adapter.factory.GenericAdapterFactory;
-import org.apache.asterix.external.api.IAdapterFactory;
-import org.apache.asterix.external.api.IDataSourceAdapter;
+import org.apache.asterix.external.api.ITypedAdapterFactory;
import org.apache.asterix.external.dataset.adapter.AdapterIdentifier;
import org.apache.asterix.external.indexing.ExternalFile;
import org.apache.asterix.metadata.IDatasetDetails;
@@ -294,7 +294,8 @@
private static DatasourceAdapter getAdapter(String adapterFactoryClassName) throws AlgebricksException {
try {
- String adapterName = ((IAdapterFactory) (Class.forName(adapterFactoryClassName).newInstance())).getAlias();
+ String adapterName =
+ ((ITypedAdapterFactory) (Class.forName(adapterFactoryClassName).newInstance())).getAlias();
return new DatasourceAdapter(new AdapterIdentifier(MetadataConstants.METADATA_DATAVERSE_NAME, adapterName),
adapterFactoryClassName, IDataSourceAdapter.AdapterType.INTERNAL);
} catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
index d9309d9..f8148ab 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/bootstrap/MetadataRecordTypes.java
@@ -68,13 +68,14 @@
public static final String FIELD_NAME_INDEX_STRUCTURE = "IndexStructure";
public static final String FIELD_NAME_INTERNAL_DETAILS = "InternalDetails";
public static final String FIELD_NAME_IS_ANONYMOUS = "IsAnonymous";
+ public static final String FIELD_NAME_IS_MISSABLE = "IsMissable";
public static final String FIELD_NAME_IS_NULLABLE = "IsNullable";
public static final String FIELD_NAME_IS_OPEN = "IsOpen";
public static final String FIELD_NAME_IS_PRIMARY = "IsPrimary";
public static final String FIELD_NAME_KIND = "Kind";
public static final String FIELD_NAME_LANGUAGE = "Language";
public static final String FIELD_NAME_LAST_REFRESH_TIME = "LastRefreshTime";
- public static final String FIELD_NAME_METADATA_DATAVERSE = "MetatypeDataverseName";
+ public static final String FIELD_NAME_METATYPE_DATAVERSE = "MetatypeDataverseName";
public static final String FIELD_NAME_METATYPE_NAME = "MetatypeName";
public static final String FIELD_NAME_NAME = "Name";
public static final String FIELD_NAME_NODE_NAME = "NodeName";
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 62cce05..715092a 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
@@ -18,12 +18,15 @@
*/
package org.apache.asterix.metadata.declared;
+import static org.apache.asterix.external.util.ExternalDataConstants.KEY_EXTERNAL_SCAN_BUFFER_SIZE;
+
import java.util.List;
+import java.util.Map;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.external.api.IAdapterFactory;
+import org.apache.asterix.external.api.ITypedAdapterFactory;
import org.apache.asterix.metadata.IDatasetDetails;
import org.apache.asterix.metadata.MetadataManager;
import org.apache.asterix.metadata.entities.Dataset;
@@ -42,6 +45,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
@@ -111,8 +115,12 @@
externalDataset.getItemTypeDataverseName(), itemTypeName).getDatatype();
ExternalDatasetDetails edd = (ExternalDatasetDetails) externalDataset.getDatasetDetails();
- IAdapterFactory adapterFactory = metadataProvider.getConfiguredAdapterFactory(externalDataset,
- edd.getAdapter(), edd.getProperties(), (ARecordType) itemType, null);
+ PhysicalOptimizationConfig physicalOptimizationConfig = context.getPhysicalOptimizationConfig();
+ int externalScanBufferSize = physicalOptimizationConfig.getExternalScanBufferSize();
+ Map<String, String> properties = edd.getProperties();
+ properties.put(KEY_EXTERNAL_SCAN_BUFFER_SIZE, String.valueOf(externalScanBufferSize));
+ ITypedAdapterFactory adapterFactory = metadataProvider.getConfiguredAdapterFactory(externalDataset,
+ edd.getAdapter(), properties, (ARecordType) itemType, null, context.getWarningCollector());
return metadataProvider.buildExternalDatasetDataScannerRuntime(jobSpec, itemType, adapterFactory);
case INTERNAL:
DataSourceId id = getId();
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSourceFactory.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSourceFactory.java
index 1e0ddbf..5503cb0 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSourceFactory.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/FunctionDataSourceFactory.java
@@ -30,6 +30,7 @@
import org.apache.hyracks.api.application.IServiceContext;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
public class FunctionDataSourceFactory implements IRecordReaderFactory<char[]> {
@@ -51,7 +52,7 @@
}
@Override
- public void configure(IServiceContext ctx, Map<String, String> configuration)
+ public void configure(IServiceContext ctx, Map<String, String> configuration, IWarningCollector warningCollector)
throws AlgebricksException, HyracksDataException {
// No Op
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
index 3460a46..e25ea22 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/LoadableDataSource.java
@@ -26,7 +26,7 @@
import org.apache.asterix.common.exceptions.AsterixException;
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.external.api.IAdapterFactory;
+import org.apache.asterix.external.api.ITypedAdapterFactory;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.InternalDatasetDetails;
import org.apache.asterix.om.types.ARecordType;
@@ -137,8 +137,8 @@
}
LoadableDataSource alds = (LoadableDataSource) dataSource;
ARecordType itemType = (ARecordType) alds.getLoadedType();
- IAdapterFactory adapterFactory = metadataProvider.getConfiguredAdapterFactory(alds.getTargetDataset(),
- alds.getAdapter(), alds.getAdapterProperties(), itemType, null);
+ ITypedAdapterFactory adapterFactory = metadataProvider.getConfiguredAdapterFactory(alds.getTargetDataset(),
+ alds.getAdapter(), alds.getAdapterProperties(), itemType, null, context.getWarningCollector());
RecordDescriptor rDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
return metadataProvider.buildLoadableDatasetScan(jobSpec, adapterFactory, rDesc);
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
index e819d65..6317d94 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataManagerUtil.java
@@ -23,6 +23,8 @@
import org.apache.asterix.common.cluster.IClusterStateManager;
import org.apache.asterix.common.config.DatasetConfig.DatasetType;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.metadata.MetadataManager;
import org.apache.asterix.metadata.MetadataTransactionContext;
import org.apache.asterix.metadata.entities.Dataset;
@@ -48,14 +50,20 @@
public static IAType findType(MetadataTransactionContext mdTxnCtx, String dataverse, String typeName)
throws AlgebricksException {
+ Datatype type = findTypeEntity(mdTxnCtx, dataverse, typeName);
+ return type != null ? type.getDatatype() : null;
+ }
+
+ public static Datatype findTypeEntity(MetadataTransactionContext mdTxnCtx, String dataverse, String typeName)
+ throws AlgebricksException {
if (dataverse == null || typeName == null) {
return null;
}
Datatype type = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataverse, typeName);
if (type == null) {
- throw new AlgebricksException("Type name '" + typeName + "' unknown in dataverse '" + dataverse + "'");
+ throw new AsterixException(ErrorCode.UNKNOWN_TYPE, dataverse + "." + typeName);
}
- return type.getDatatype();
+ return type;
}
public static ARecordType findOutputRecordType(MetadataTransactionContext mdTxnCtx, String dataverse,
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 0a72ceb..ee6f4d8 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
@@ -39,6 +39,7 @@
import org.apache.asterix.common.dataflow.ICcApplicationContext;
import org.apache.asterix.common.dataflow.LSMTreeInsertDeleteOperatorDescriptor;
import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.external.IDataSourceAdapter;
import org.apache.asterix.common.metadata.LockList;
import org.apache.asterix.common.storage.ICompressionManager;
import org.apache.asterix.common.transactions.ITxnIdFactory;
@@ -48,8 +49,7 @@
import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
import org.apache.asterix.dataflow.data.nontagged.serde.SerializerDeserializerUtil;
import org.apache.asterix.external.adapter.factory.LookupAdapterFactory;
-import org.apache.asterix.external.api.IAdapterFactory;
-import org.apache.asterix.external.api.IDataSourceAdapter;
+import org.apache.asterix.external.api.ITypedAdapterFactory;
import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
import org.apache.asterix.external.indexing.ExternalFile;
import org.apache.asterix.external.indexing.IndexingConstants;
@@ -72,6 +72,7 @@
import org.apache.asterix.metadata.dataset.hints.DatasetHints.DatasetCardinalityHint;
import org.apache.asterix.metadata.entities.Dataset;
import org.apache.asterix.metadata.entities.DatasourceAdapter;
+import org.apache.asterix.metadata.entities.Datatype;
import org.apache.asterix.metadata.entities.Dataverse;
import org.apache.asterix.metadata.entities.ExternalDatasetDetails;
import org.apache.asterix.metadata.entities.Feed;
@@ -129,6 +130,7 @@
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.dataflow.value.ITypeTraits;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
import org.apache.hyracks.api.io.FileSplit;
import org.apache.hyracks.api.job.JobSpecification;
import org.apache.hyracks.api.result.IResultMetadata;
@@ -345,6 +347,10 @@
return MetadataManagerUtil.findNodes(mdTxnCtx, nodeGroupName);
}
+ public Datatype findTypeEntity(String dataverse, String typeName) throws AlgebricksException {
+ return MetadataManagerUtil.findTypeEntity(mdTxnCtx, dataverse, typeName);
+ }
+
public IAType findType(String dataverse, String typeName) throws AlgebricksException {
return MetadataManagerUtil.findType(mdTxnCtx, dataverse, typeName);
}
@@ -416,7 +422,7 @@
}
protected Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildLoadableDatasetScan(
- JobSpecification jobSpec, IAdapterFactory adapterFactory, RecordDescriptor rDesc)
+ JobSpecification jobSpec, ITypedAdapterFactory adapterFactory, RecordDescriptor rDesc)
throws AlgebricksException {
ExternalScanOperatorDescriptor dataScanner = new ExternalScanOperatorDescriptor(jobSpec, rDesc, adapterFactory);
try {
@@ -430,14 +436,14 @@
return MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
}
- public Triple<IOperatorDescriptor, AlgebricksPartitionConstraint, IAdapterFactory> buildFeedIntakeRuntime(
+ public Triple<IOperatorDescriptor, AlgebricksPartitionConstraint, ITypedAdapterFactory> buildFeedIntakeRuntime(
JobSpecification jobSpec, Feed feed, FeedPolicyAccessor policyAccessor) throws Exception {
- Triple<IAdapterFactory, RecordDescriptor, IDataSourceAdapter.AdapterType> factoryOutput;
+ Triple<ITypedAdapterFactory, RecordDescriptor, IDataSourceAdapter.AdapterType> factoryOutput;
factoryOutput =
FeedMetadataUtil.getFeedFactoryAndOutput(feed, policyAccessor, mdTxnCtx, getApplicationContext());
ARecordType recordType =
FeedMetadataUtil.getOutputType(feed, feed.getConfiguration().get(ExternalDataConstants.KEY_TYPE_NAME));
- IAdapterFactory adapterFactory = factoryOutput.first;
+ ITypedAdapterFactory adapterFactory = factoryOutput.first;
FeedIntakeOperatorDescriptor feedIngestor = null;
switch (factoryOutput.third) {
case INTERNAL:
@@ -775,12 +781,14 @@
return numElementsHint / numPartitions;
}
- protected IAdapterFactory getConfiguredAdapterFactory(Dataset dataset, String adapterName,
- Map<String, String> configuration, ARecordType itemType, ARecordType metaType) throws AlgebricksException {
+ protected ITypedAdapterFactory getConfiguredAdapterFactory(Dataset dataset, String adapterName,
+ Map<String, String> configuration, ARecordType itemType, ARecordType metaType,
+ IWarningCollector warningCollector) throws AlgebricksException {
try {
- configuration.put(ExternalDataConstants.KEY_DATAVERSE, dataset.getDataverseName());
- IAdapterFactory adapterFactory = AdapterFactoryProvider.getAdapterFactory(
- getApplicationContext().getServiceContext(), adapterName, configuration, itemType, metaType);
+ configuration.put(ExternalDataConstants.KEY_DATASET_DATAVERSE, dataset.getDataverseName());
+ ITypedAdapterFactory adapterFactory =
+ AdapterFactoryProvider.getAdapterFactory(getApplicationContext().getServiceContext(), adapterName,
+ configuration, itemType, metaType, warningCollector);
// check to see if dataset is indexed
Index filesIndex =
@@ -853,7 +861,7 @@
ExternalDatasetDetails datasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
LookupAdapterFactory<?> adapterFactory = AdapterFactoryProvider.getLookupAdapterFactory(
getApplicationContext().getServiceContext(), datasetDetails.getProperties(), itemType, ridIndexes,
- retainInput, retainMissing, context.getMissingWriterFactory());
+ retainInput, retainMissing, context.getMissingWriterFactory(), context.getWarningCollector());
String fileIndexName = IndexingConstants.getFilesIndexName(dataset.getDatasetName());
Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc =
metadataProvider.getSplitProviderAndConstraints(dataset, fileIndexName);
@@ -922,7 +930,7 @@
}
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDatasetDataScannerRuntime(
- JobSpecification jobSpec, IAType itemType, IAdapterFactory adapterFactory) throws AlgebricksException {
+ JobSpecification jobSpec, IAType itemType, ITypedAdapterFactory adapterFactory) throws AlgebricksException {
if (itemType.getTypeTag() != ATypeTag.OBJECT) {
throw new AlgebricksException("Can only scan datasets of records.");
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java
index 14f76eb..d2284b9 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/BuiltinTypeMap.java
@@ -28,7 +28,6 @@
import org.apache.asterix.common.exceptions.MetadataException;
import org.apache.asterix.common.transactions.TxnId;
import org.apache.asterix.metadata.MetadataNode;
-import org.apache.asterix.om.types.AUnionType;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -95,7 +94,7 @@
}
public static IAType getTypeFromTypeName(MetadataNode metadataNode, TxnId txnId, String dataverseName,
- String typeName, boolean optional) throws AlgebricksException {
+ String typeName) throws AlgebricksException {
IAType type = _builtinTypeMap.get(typeName);
if (type == null) {
try {
@@ -105,9 +104,6 @@
throw new MetadataException(e);
}
}
- if (optional) {
- type = AUnionType.createUnknownableType(type);
- }
return type;
}
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/DatasourceAdapter.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/DatasourceAdapter.java
index b72c058..c29fb93 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/DatasourceAdapter.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/DatasourceAdapter.java
@@ -18,7 +18,7 @@
*/
package org.apache.asterix.metadata.entities;
-import org.apache.asterix.external.api.IDataSourceAdapter.AdapterType;
+import org.apache.asterix.common.external.IDataSourceAdapter.AdapterType;
import org.apache.asterix.external.dataset.adapter.AdapterIdentifier;
import org.apache.asterix.metadata.MetadataCache;
import org.apache.asterix.metadata.api.IMetadataEntity;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
index 7f8b9bf6..74f5076 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
@@ -25,6 +25,7 @@
import java.io.DataOutput;
import java.util.ArrayList;
import java.util.Calendar;
+import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.LinkedHashMap;
@@ -68,6 +69,7 @@
import org.apache.asterix.om.types.IAType;
import org.apache.asterix.runtime.compression.CompressionManager;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -135,20 +137,9 @@
String nodeGroupName =
((AString) datasetRecord.getValueByPos(MetadataRecordTypes.DATASET_ARECORD_GROUPNAME_FIELD_INDEX))
.getStringValue();
- String compactionPolicy = ((AString) datasetRecord
- .getValueByPos(MetadataRecordTypes.DATASET_ARECORD_COMPACTION_POLICY_FIELD_INDEX)).getStringValue();
- IACursor cursor = ((AOrderedList) datasetRecord
- .getValueByPos(MetadataRecordTypes.DATASET_ARECORD_COMPACTION_POLICY_PROPERTIES_FIELD_INDEX))
- .getCursor();
- Map<String, String> compactionPolicyProperties = new LinkedHashMap<>();
- String key;
- String value;
- while (cursor.next()) {
- ARecord field = (ARecord) cursor.get();
- key = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_NAME_FIELD_INDEX)).getStringValue();
- value = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_VALUE_FIELD_INDEX)).getStringValue();
- compactionPolicyProperties.put(key, value);
- }
+
+ Pair<String, Map<String, String>> compactionPolicy = readCompactionPolicy(datasetType, datasetRecord);
+
switch (datasetType) {
case INTERNAL: {
ARecord datasetDetailsRecord = (ARecord) datasetRecord
@@ -159,7 +150,7 @@
PartitioningStrategy partitioningStrategy = PartitioningStrategy.valueOf(((AString) datasetDetailsRecord
.getValueByPos(MetadataRecordTypes.INTERNAL_DETAILS_ARECORD_PARTITIONSTRATEGY_FIELD_INDEX))
.getStringValue());
- cursor = ((AOrderedList) datasetDetailsRecord
+ IACursor cursor = ((AOrderedList) datasetDetailsRecord
.getValueByPos(MetadataRecordTypes.INTERNAL_DETAILS_ARECORD_PARTITIONKEY_FIELD_INDEX))
.getCursor();
List<List<String>> partitioningKey = new ArrayList<>();
@@ -220,15 +211,15 @@
String adapter = ((AString) datasetDetailsRecord
.getValueByPos(MetadataRecordTypes.EXTERNAL_DETAILS_ARECORD_DATASOURCE_ADAPTER_FIELD_INDEX))
.getStringValue();
- cursor = ((AOrderedList) datasetDetailsRecord
+ IACursor cursor = ((AOrderedList) datasetDetailsRecord
.getValueByPos(MetadataRecordTypes.EXTERNAL_DETAILS_ARECORD_PROPERTIES_FIELD_INDEX))
.getCursor();
Map<String, String> properties = new HashMap<>();
while (cursor.next()) {
ARecord field = (ARecord) cursor.get();
- key = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_NAME_FIELD_INDEX))
+ String key = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_NAME_FIELD_INDEX))
.getStringValue();
- value = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_VALUE_FIELD_INDEX))
+ String value = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_VALUE_FIELD_INDEX))
.getStringValue();
properties.put(key, value);
}
@@ -250,7 +241,7 @@
String metaTypeDataverseName = null;
String metaTypeName = null;
int metaTypeDataverseNameIndex =
- datasetRecord.getType().getFieldIndex(MetadataRecordTypes.FIELD_NAME_METADATA_DATAVERSE);
+ datasetRecord.getType().getFieldIndex(MetadataRecordTypes.FIELD_NAME_METATYPE_DATAVERSE);
if (metaTypeDataverseNameIndex >= 0) {
metaTypeDataverseName =
((AString) datasetRecord.getValueByPos(metaTypeDataverseNameIndex)).getStringValue();
@@ -262,10 +253,34 @@
String compressionScheme = getCompressionScheme(datasetRecord);
return new Dataset(dataverseName, datasetName, typeDataverseName, typeName, metaTypeDataverseName, metaTypeName,
- nodeGroupName, compactionPolicy, compactionPolicyProperties, datasetDetails, hints, datasetType,
+ nodeGroupName, compactionPolicy.first, compactionPolicy.second, datasetDetails, hints, datasetType,
datasetId, pendingOp, rebalanceCount, compressionScheme);
}
+ protected Pair<String, Map<String, String>> readCompactionPolicy(DatasetType datasetType, ARecord datasetRecord) {
+
+ String compactionPolicy = ((AString) datasetRecord
+ .getValueByPos(MetadataRecordTypes.DATASET_ARECORD_COMPACTION_POLICY_FIELD_INDEX)).getStringValue();
+ AOrderedList compactionPolicyPropertiesList = ((AOrderedList) datasetRecord
+ .getValueByPos(MetadataRecordTypes.DATASET_ARECORD_COMPACTION_POLICY_PROPERTIES_FIELD_INDEX));
+
+ Map<String, String> compactionPolicyProperties;
+ if (compactionPolicyPropertiesList.size() > 0) {
+ compactionPolicyProperties = new LinkedHashMap<>();
+ for (IACursor cursor = compactionPolicyPropertiesList.getCursor(); cursor.next();) {
+ ARecord field = (ARecord) cursor.get();
+ String key = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_NAME_FIELD_INDEX))
+ .getStringValue();
+ String value = ((AString) field.getValueByPos(MetadataRecordTypes.PROPERTIES_VALUE_FIELD_INDEX))
+ .getStringValue();
+ compactionPolicyProperties.put(key, value);
+ }
+ } else {
+ compactionPolicyProperties = Collections.emptyMap();
+ }
+ return new Pair<>(compactionPolicy, compactionPolicyProperties);
+ }
+
private long getRebalanceCount(ARecord datasetRecord) {
// Read the rebalance count if there is one.
int rebalanceCountIndex =
@@ -342,29 +357,9 @@
stringSerde.serialize(aString, fieldValue.getDataOutput());
recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_GROUPNAME_FIELD_INDEX, fieldValue);
- // write field 6
- fieldValue.reset();
- aString.setValue(dataset.getCompactionPolicy());
- stringSerde.serialize(aString, fieldValue.getDataOutput());
- recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_COMPACTION_POLICY_FIELD_INDEX, fieldValue);
-
- // write field 7
- listBuilder.reset((AOrderedListType) MetadataRecordTypes.DATASET_RECORDTYPE
- .getFieldTypes()[MetadataRecordTypes.DATASET_ARECORD_COMPACTION_POLICY_PROPERTIES_FIELD_INDEX]);
- if (dataset.getCompactionPolicyProperties() != null) {
- for (Map.Entry<String, String> property : dataset.getCompactionPolicyProperties().entrySet()) {
- String name = property.getKey();
- String value = property.getValue();
- itemValue.reset();
- DatasetUtil.writePropertyTypeRecord(name, value, itemValue.getDataOutput(),
- MetadataRecordTypes.COMPACTION_POLICY_PROPERTIES_RECORDTYPE);
- listBuilder.addItem(itemValue);
- }
- }
- fieldValue.reset();
- listBuilder.write(fieldValue.getDataOutput(), true);
- recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_COMPACTION_POLICY_PROPERTIES_FIELD_INDEX,
- fieldValue);
+ // write field 6/7
+ writeCompactionPolicy(dataset.getDatasetType(), dataset.getCompactionPolicy(),
+ dataset.getCompactionPolicyProperties(), listBuilder, itemValue);
// write field 8/9
fieldValue.reset();
@@ -414,6 +409,34 @@
return tuple;
}
+ protected void writeCompactionPolicy(DatasetType datasetType, String compactionPolicy,
+ Map<String, String> compactionPolicyProperties, OrderedListBuilder listBuilder,
+ ArrayBackedValueStorage itemValue) throws HyracksDataException {
+ // write field 6
+ fieldValue.reset();
+ aString.setValue(compactionPolicy);
+ stringSerde.serialize(aString, fieldValue.getDataOutput());
+ recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_COMPACTION_POLICY_FIELD_INDEX, fieldValue);
+
+ // write field 7
+ listBuilder.reset((AOrderedListType) MetadataRecordTypes.DATASET_RECORDTYPE
+ .getFieldTypes()[MetadataRecordTypes.DATASET_ARECORD_COMPACTION_POLICY_PROPERTIES_FIELD_INDEX]);
+ if (compactionPolicyProperties != null && !compactionPolicyProperties.isEmpty()) {
+ for (Map.Entry<String, String> property : compactionPolicyProperties.entrySet()) {
+ String name = property.getKey();
+ String value = property.getValue();
+ itemValue.reset();
+ DatasetUtil.writePropertyTypeRecord(name, value, itemValue.getDataOutput(),
+ MetadataRecordTypes.COMPACTION_POLICY_PROPERTIES_RECORDTYPE);
+ listBuilder.addItem(itemValue);
+ }
+ }
+ fieldValue.reset();
+ listBuilder.write(fieldValue.getDataOutput(), true);
+ recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_COMPACTION_POLICY_PROPERTIES_FIELD_INDEX,
+ fieldValue);
+ }
+
/**
* Keep protected to allow other extensions to add additional fields
*
@@ -430,7 +453,7 @@
if (dataset.hasMetaPart()) {
// write open field 1, the meta item type Dataverse name.
fieldName.reset();
- aString.setValue(MetadataRecordTypes.FIELD_NAME_METADATA_DATAVERSE);
+ aString.setValue(MetadataRecordTypes.FIELD_NAME_METATYPE_DATAVERSE);
stringSerde.serialize(aString, fieldName.getDataOutput());
fieldValue.reset();
aString.setValue(dataset.getMetaItemTypeDataverseName());
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java
index 8f630cf..9e65c08 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatasourceAdapterTupleTranslator.java
@@ -24,7 +24,7 @@
import java.io.DataInputStream;
import java.util.Calendar;
-import org.apache.asterix.external.api.IDataSourceAdapter;
+import org.apache.asterix.common.external.IDataSourceAdapter;
import org.apache.asterix.external.dataset.adapter.AdapterIdentifier;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
index 32951d5..8e9118e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
@@ -36,6 +36,7 @@
import org.apache.asterix.metadata.bootstrap.MetadataRecordTypes;
import org.apache.asterix.metadata.entities.BuiltinTypeMap;
import org.apache.asterix.metadata.entities.Datatype;
+import org.apache.asterix.metadata.utils.TypeUtil;
import org.apache.asterix.om.base.ABoolean;
import org.apache.asterix.om.base.AOrderedList;
import org.apache.asterix.om.base.ARecord;
@@ -49,7 +50,6 @@
import org.apache.asterix.om.types.AbstractCollectionType;
import org.apache.asterix.om.types.AbstractComplexType;
import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.utils.NonTaggedFormatUtil;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.exceptions.ErrorCode;
@@ -82,6 +82,7 @@
SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(MetadataRecordTypes.DATATYPE_RECORDTYPE);
private final MetadataNode metadataNode;
private final TxnId txnId;
+ protected final transient ArrayBackedValueStorage fieldName = new ArrayBackedValueStorage();
protected DatatypeTupleTranslator(TxnId txnId, MetadataNode metadataNode, boolean getTuple) {
super(getTuple, MetadataPrimaryIndexes.DATATYPE_DATASET.getFieldCount());
@@ -123,8 +124,7 @@
ARecord recordType = (ARecord) derivedTypeRecord
.getValueByPos(MetadataRecordTypes.DERIVEDTYPE_ARECORD_RECORD_FIELD_INDEX);
boolean isOpen = ((ABoolean) recordType
- .getValueByPos(MetadataRecordTypes.RECORDTYPE_ARECORD_ISOPEN_FIELD_INDEX)).getBoolean()
- .booleanValue();
+ .getValueByPos(MetadataRecordTypes.RECORDTYPE_ARECORD_ISOPEN_FIELD_INDEX)).getBoolean();
int numberOfFields = ((AOrderedList) recordType
.getValueByPos(MetadataRecordTypes.RECORDTYPE_ARECORD_FIELDS_FIELD_INDEX)).size();
IACursor cursor = ((AOrderedList) recordType
@@ -141,11 +141,24 @@
fieldTypeName =
((AString) field.getValueByPos(MetadataRecordTypes.FIELD_ARECORD_FIELDTYPE_FIELD_INDEX))
.getStringValue();
+
boolean isNullable = ((ABoolean) field
- .getValueByPos(MetadataRecordTypes.FIELD_ARECORD_ISNULLABLE_FIELD_INDEX)).getBoolean()
- .booleanValue();
- fieldTypes[fieldId] = BuiltinTypeMap.getTypeFromTypeName(metadataNode, txnId, dataverseName,
- fieldTypeName, isNullable);
+ .getValueByPos(MetadataRecordTypes.FIELD_ARECORD_ISNULLABLE_FIELD_INDEX)).getBoolean();
+
+ int isMissableIdx = field.getType().getFieldIndex(MetadataRecordTypes.FIELD_NAME_IS_MISSABLE);
+ boolean isMissable;
+ if (isMissableIdx >= 0) {
+ isMissable = ((ABoolean) field.getValueByPos(isMissableIdx)).getBoolean();
+ } else {
+ // back-compat
+ // we previously stored 'isNullable' = true if type was 'unknowable',
+ // or 'isNullable' = 'false' if the type was 'not unknowable'.
+ isMissable = isNullable;
+ }
+
+ IAType fieldType =
+ BuiltinTypeMap.getTypeFromTypeName(metadataNode, txnId, dataverseName, fieldTypeName);
+ fieldTypes[fieldId] = TypeUtil.createQuantifiedType(fieldType, isNullable, isMissable);
fieldId++;
}
return new Datatype(dataverseName, datatypeName,
@@ -157,17 +170,16 @@
.getStringValue();
return new Datatype(dataverseName, datatypeName,
new AUnorderedListType(BuiltinTypeMap.getTypeFromTypeName(metadataNode, txnId,
- dataverseName, unorderedlistTypeName, false), datatypeName),
+ dataverseName, unorderedlistTypeName), datatypeName),
isAnonymous);
}
case ORDEREDLIST: {
String orderedlistTypeName = ((AString) derivedTypeRecord
.getValueByPos(MetadataRecordTypes.DERIVEDTYPE_ARECORD_ORDEREDLIST_FIELD_INDEX))
.getStringValue();
- return new Datatype(dataverseName, datatypeName,
- new AOrderedListType(BuiltinTypeMap.getTypeFromTypeName(metadataNode, txnId, dataverseName,
- orderedlistTypeName, false), datatypeName),
- isAnonymous);
+ return new Datatype(dataverseName, datatypeName, new AOrderedListType(
+ BuiltinTypeMap.getTypeFromTypeName(metadataNode, txnId, dataverseName, orderedlistTypeName),
+ datatypeName), isAnonymous);
}
default:
throw new UnsupportedOperationException("Unsupported derived type: " + tag);
@@ -203,20 +215,15 @@
stringSerde.serialize(aString, fieldValue.getDataOutput());
recordBuilder.addField(MetadataRecordTypes.DATATYPE_ARECORD_DATATYPENAME_FIELD_INDEX, fieldValue);
+ // write field 2
IAType fieldType = dataType.getDatatype();
- // unwrap nullable type out of the union
- if (fieldType.getTypeTag() == ATypeTag.UNION) {
- fieldType = ((AUnionType) dataType.getDatatype()).getActualType();
- }
-
- // write field 3
if (fieldType.getTypeTag().isDerivedType()) {
fieldValue.reset();
writeDerivedTypeRecord(dataType, (AbstractComplexType) fieldType, fieldValue.getDataOutput());
recordBuilder.addField(MetadataRecordTypes.DATATYPE_ARECORD_DERIVED_FIELD_INDEX, fieldValue);
}
- // write field 4
+ // write field 3
fieldValue.reset();
aString.setValue(Calendar.getInstance().getTime().toString());
stringSerde.serialize(aString, fieldValue.getDataOutput());
@@ -232,7 +239,7 @@
private void writeDerivedTypeRecord(Datatype type, AbstractComplexType derivedDatatype, DataOutput out)
throws HyracksDataException {
- DerivedTypeTag tag = null;
+ DerivedTypeTag tag;
IARecordBuilder derivedRecordBuilder = new RecordBuilder();
ArrayBackedValueStorage fieldValue = new ArrayBackedValueStorage();
switch (derivedDatatype.getTypeTag()) {
@@ -260,7 +267,7 @@
// write field 1
fieldValue.reset();
- booleanSerde.serialize(type.getIsAnonymous() ? ABoolean.TRUE : ABoolean.FALSE, fieldValue.getDataOutput());
+ booleanSerde.serialize(ABoolean.valueOf(type.getIsAnonymous()), fieldValue.getDataOutput());
derivedRecordBuilder.addField(MetadataRecordTypes.DERIVEDTYPE_ARECORD_ISANONYMOUS_FIELD_INDEX, fieldValue);
switch (tag) {
@@ -308,14 +315,18 @@
ARecordType recType = (ARecordType) type;
OrderedListBuilder listBuilder = new OrderedListBuilder();
listBuilder.reset(new AOrderedListType(MetadataRecordTypes.FIELD_RECORDTYPE, null));
- IAType fieldType = null;
+ IAType fieldType;
for (int i = 0; i < recType.getFieldNames().length; i++) {
fieldType = recType.getFieldTypes()[i];
boolean fieldIsNullable = false;
- if (NonTaggedFormatUtil.isOptional(fieldType)) {
- fieldIsNullable = true;
- fieldType = ((AUnionType) fieldType).getActualType();
+ boolean fieldIsMissable = false;
+
+ if (fieldType.getTypeTag() == ATypeTag.UNION) {
+ AUnionType fieldUnionType = (AUnionType) fieldType;
+ fieldIsNullable = fieldUnionType.isNullableType();
+ fieldIsMissable = fieldUnionType.isMissableType();
+ fieldType = fieldUnionType.getActualType();
}
if (fieldType.getTypeTag().isDerivedType()) {
handleNestedDerivedType(fieldType.getTypeName(), (AbstractComplexType) fieldType, instance,
@@ -342,6 +353,14 @@
booleanSerde.serialize(fieldIsNullable ? ABoolean.TRUE : ABoolean.FALSE, fieldValue.getDataOutput());
fieldRecordBuilder.addField(MetadataRecordTypes.FIELD_ARECORD_ISNULLABLE_FIELD_INDEX, fieldValue);
+ // write open fields
+ fieldName.reset();
+ aString.setValue(MetadataRecordTypes.FIELD_NAME_IS_MISSABLE);
+ stringSerde.serialize(aString, fieldName.getDataOutput());
+ fieldValue.reset();
+ booleanSerde.serialize(ABoolean.valueOf(fieldIsMissable), fieldValue.getDataOutput());
+ fieldRecordBuilder.addField(fieldName, fieldValue);
+
// write record
fieldRecordBuilder.write(itemValue.getDataOutput(), true);
@@ -376,7 +395,7 @@
throw HyracksDataException.create(e);
} else {
HyracksDataException hde = (HyracksDataException) e.getCause();
- if (!hde.getComponent().equals(ErrorCode.HYRACKS) || hde.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!hde.matches(ErrorCode.DUPLICATE_KEY)) {
throw hde;
}
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
index a154d7f..0c56c5e 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
@@ -141,7 +141,7 @@
List<IAType> searchKeyType = new ArrayList<>(searchKey.size());
while (fieldTypeCursor.next()) {
String typeName = ((AString) fieldTypeCursor.get()).getStringValue();
- IAType fieldType = BuiltinTypeMap.getTypeFromTypeName(metadataNode, txnId, dvName, typeName, false);
+ IAType fieldType = BuiltinTypeMap.getTypeFromTypeName(metadataNode, txnId, dvName, typeName);
searchKeyType.add(fieldType);
}
boolean isOverridingKeyTypes = !searchKeyType.isEmpty();
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
index 3ae0fec..792501b 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/feeds/FeedMetadataUtil.java
@@ -28,9 +28,10 @@
import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.exceptions.MetadataException;
-import org.apache.asterix.external.api.IAdapterFactory;
-import org.apache.asterix.external.api.IDataSourceAdapter;
-import org.apache.asterix.external.api.IDataSourceAdapter.AdapterType;
+import org.apache.asterix.common.exceptions.NoOpWarningCollector;
+import org.apache.asterix.common.external.IDataSourceAdapter;
+import org.apache.asterix.common.external.IDataSourceAdapter.AdapterType;
+import org.apache.asterix.external.api.ITypedAdapterFactory;
import org.apache.asterix.external.feed.api.IFeed;
import org.apache.asterix.external.feed.policy.FeedPolicyAccessor;
import org.apache.asterix.external.provider.AdapterFactoryProvider;
@@ -52,6 +53,7 @@
import org.apache.hyracks.algebricks.common.utils.Triple;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
/**
* A utility class for providing helper functions for feeds TODO: Refactor this
@@ -98,8 +100,8 @@
return feedPolicy;
}
- public static void validateFeed(Feed feed, MetadataTransactionContext mdTxnCtx, ICcApplicationContext appCtx)
- throws AlgebricksException {
+ public static void validateFeed(Feed feed, MetadataTransactionContext mdTxnCtx, ICcApplicationContext appCtx,
+ IWarningCollector warningCollector) throws AlgebricksException {
try {
Map<String, String> configuration = feed.getConfiguration();
ARecordType adapterOutputType = getOutputType(feed, configuration.get(ExternalDataConstants.KEY_TYPE_NAME));
@@ -117,31 +119,31 @@
adapterEntity = MetadataManager.INSTANCE.getAdapter(mdTxnCtx, feed.getDataverseName(), adapterName);
}
AdapterType adapterType;
- IAdapterFactory adapterFactory;
+ ITypedAdapterFactory adapterFactory;
if (adapterEntity != null) {
adapterType = adapterEntity.getType();
String adapterFactoryClassname = adapterEntity.getClassname();
switch (adapterType) {
case INTERNAL:
- adapterFactory = (IAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
+ adapterFactory = (ITypedAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
break;
case EXTERNAL:
String[] anameComponents = adapterName.split("#");
String libraryName = anameComponents[0];
ClassLoader cl =
appCtx.getLibraryManager().getLibraryClassLoader(feed.getDataverseName(), libraryName);
- adapterFactory = (IAdapterFactory) cl.loadClass(adapterFactoryClassname).newInstance();
+ adapterFactory = (ITypedAdapterFactory) cl.loadClass(adapterFactoryClassname).newInstance();
break;
default:
throw new AsterixException("Unknown Adapter type " + adapterType);
}
- adapterFactory.setOutputType(adapterOutputType);
- adapterFactory.setMetaType(metaType);
- adapterFactory.configure(appCtx.getServiceContext(), configuration);
} else {
- AdapterFactoryProvider.getAdapterFactory(appCtx.getServiceContext(), adapterName, configuration,
- adapterOutputType, metaType);
+ ExternalDataUtils.prepare(adapterName, configuration);
+ adapterFactory = (ITypedAdapterFactory) appCtx.getAdapterFactoryService().createAdapterFactory();
}
+ adapterFactory.setOutputType(adapterOutputType);
+ adapterFactory.setMetaType(metaType);
+ adapterFactory.configure(appCtx.getServiceContext(), configuration, warningCollector);
if (metaType == null && configuration.containsKey(ExternalDataConstants.KEY_META_TYPE_NAME)) {
metaType = getOutputType(feed, configuration.get(ExternalDataConstants.KEY_META_TYPE_NAME));
if (metaType == null) {
@@ -165,17 +167,17 @@
}
@SuppressWarnings("rawtypes")
- public static Triple<IAdapterFactory, RecordDescriptor, AdapterType> getFeedFactoryAndOutput(Feed feed,
+ public static Triple<ITypedAdapterFactory, RecordDescriptor, AdapterType> getFeedFactoryAndOutput(Feed feed,
FeedPolicyAccessor policyAccessor, MetadataTransactionContext mdTxnCtx, ICcApplicationContext appCtx)
throws AlgebricksException {
// This method needs to be re-visited
String adapterName = null;
DatasourceAdapter adapterEntity = null;
String adapterFactoryClassname = null;
- IAdapterFactory adapterFactory = null;
+ ITypedAdapterFactory adapterFactory = null;
ARecordType adapterOutputType = null;
ARecordType metaType = null;
- Triple<IAdapterFactory, RecordDescriptor, IDataSourceAdapter.AdapterType> feedProps = null;
+ Triple<ITypedAdapterFactory, RecordDescriptor, IDataSourceAdapter.AdapterType> feedProps = null;
IDataSourceAdapter.AdapterType adapterType = null;
try {
Map<String, String> configuration = feed.getConfiguration();
@@ -196,24 +198,24 @@
adapterFactoryClassname = adapterEntity.getClassname();
switch (adapterType) {
case INTERNAL:
- adapterFactory = (IAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
+ adapterFactory = (ITypedAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
break;
case EXTERNAL:
String[] anameComponents = adapterName.split("#");
String libraryName = anameComponents[0];
ClassLoader cl =
appCtx.getLibraryManager().getLibraryClassLoader(feed.getDataverseName(), libraryName);
- adapterFactory = (IAdapterFactory) cl.loadClass(adapterFactoryClassname).newInstance();
+ adapterFactory = (ITypedAdapterFactory) cl.loadClass(adapterFactoryClassname).newInstance();
break;
default:
throw new AsterixException("Unknown Adapter type " + adapterType);
}
adapterFactory.setOutputType(adapterOutputType);
adapterFactory.setMetaType(metaType);
- adapterFactory.configure(appCtx.getServiceContext(), configuration);
+ adapterFactory.configure(appCtx.getServiceContext(), configuration, NoOpWarningCollector.INSTANCE);
} else {
adapterFactory = AdapterFactoryProvider.getAdapterFactory(appCtx.getServiceContext(), adapterName,
- configuration, adapterOutputType, metaType);
+ configuration, adapterOutputType, metaType, NoOpWarningCollector.INSTANCE);
adapterType = IDataSourceAdapter.AdapterType.INTERNAL;
}
if (metaType == null) {
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
index 4b7d359..a5084e0 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/DatasetUtil.java
@@ -99,6 +99,8 @@
*/
public static final byte OP_UPSERT = 0x03;
+ private static final String DATASET_INLINE_TYPE_PREFIX = "$d$t$";
+
private DatasetUtil() {
}
@@ -207,9 +209,13 @@
* field is actually a key by making sure the field is coming from the right record (data record or meta record),
* e.g. if the field name happens to be equal to the key name but the field is coming from the data record while
* the key is coming from the meta record.
- * @param keySourceIndicator indicates where the key is coming from, 1 from meta record, 0 from data record
- * @param keyIndex the key index we're checking the field against
- * @param fieldFromMeta whether the field is coming from the meta record or the data record
+ *
+ * @param keySourceIndicator
+ * indicates where the key is coming from, 1 from meta record, 0 from data record
+ * @param keyIndex
+ * the key index we're checking the field against
+ * @param fieldFromMeta
+ * whether the field is coming from the meta record or the data record
* @return true if the key source matches the field source. Otherwise, false.
*/
private static boolean keySourceMatches(List<Integer> keySourceIndicator, int keyIndex, boolean fieldFromMeta) {
@@ -603,4 +609,13 @@
}
return new Pair<>(first, second);
}
+
+ public static String createInlineTypeName(String datasetName, boolean forMetaItemType) {
+ char typeChar = forMetaItemType ? 'm' : 'i';
+ return DATASET_INLINE_TYPE_PREFIX + typeChar + '$' + datasetName;
+ }
+
+ public static boolean isInlineTypeName(Dataset dataset, String typeDataverseName, String typeName) {
+ return dataset.getDataverseName().equals(typeDataverseName) && typeName.startsWith(DATASET_INLINE_TYPE_PREFIX);
+ }
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
index 47db3b0..9847fe4 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ExternalIndexingOperations.java
@@ -30,7 +30,8 @@
import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
import org.apache.asterix.common.config.DatasetConfig.TransactionState;
import org.apache.asterix.common.context.IStorageComponentProvider;
-import org.apache.asterix.external.api.IAdapterFactory;
+import org.apache.asterix.common.exceptions.NoOpWarningCollector;
+import org.apache.asterix.external.api.ITypedAdapterFactory;
import org.apache.asterix.external.indexing.ExternalFile;
import org.apache.asterix.external.indexing.IndexingConstants;
import org.apache.asterix.external.operators.ExternalDatasetIndexesAbortOperatorDescriptor;
@@ -254,9 +255,9 @@
throws HyracksDataException, AlgebricksException {
ExternalDatasetDetails externalDatasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
Map<String, String> configuration = externalDatasetDetails.getProperties();
- IAdapterFactory adapterFactory = AdapterFactoryProvider.getIndexingAdapterFactory(
+ ITypedAdapterFactory adapterFactory = AdapterFactoryProvider.getIndexingAdapterFactory(
metadataProvider.getApplicationContext().getServiceContext(), externalDatasetDetails.getAdapter(),
- configuration, (ARecordType) itemType, files, true, null);
+ configuration, (ARecordType) itemType, files, true, null, NoOpWarningCollector.INSTANCE);
ExternalScanOperatorDescriptor scanOp =
new ExternalScanOperatorDescriptor(jobSpec, indexerDesc, adapterFactory);
scanOp.setSourceLocation(sourceLoc);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
index baac712..037a560 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataLockUtil.java
@@ -27,10 +27,11 @@
private MetadataLockUtil() {
}
- public static void createDatasetBegin(IMetadataLockManager lockMgr, LockList locks, String dataverseName,
- String itemTypeDataverseName, String itemTypeFullyQualifiedName, String metaItemTypeDataverseName,
- String metaItemTypeFullyQualifiedName, String nodeGroupName, String compactionPolicyName,
- String datasetFullyQualifiedName, boolean isDefaultCompactionPolicy) throws AlgebricksException {
+ public static void createDatasetBeginPre(IMetadataLockManager lockMgr, LockList locks, String dataverseName,
+ String itemTypeDataverseName, String itemTypeFullyQualifiedName, boolean itemTypeAnonymous,
+ String metaItemTypeDataverseName, String metaItemTypeFullyQualifiedName, boolean metaItemTypeAnonymous,
+ String nodeGroupName, String compactionPolicyName, boolean isDefaultCompactionPolicy)
+ throws AlgebricksException {
lockMgr.acquireDataverseReadLock(locks, dataverseName);
if (!dataverseName.equals(itemTypeDataverseName)) {
lockMgr.acquireDataverseReadLock(locks, itemTypeDataverseName);
@@ -39,10 +40,20 @@
&& !metaItemTypeDataverseName.equals(itemTypeDataverseName)) {
lockMgr.acquireDataverseReadLock(locks, metaItemTypeDataverseName);
}
- lockMgr.acquireDataTypeReadLock(locks, itemTypeFullyQualifiedName);
+ if (itemTypeAnonymous) {
+ // the datatype will be created
+ lockMgr.acquireDataTypeWriteLock(locks, itemTypeFullyQualifiedName);
+ } else {
+ lockMgr.acquireDataTypeReadLock(locks, itemTypeFullyQualifiedName);
+ }
if (metaItemTypeFullyQualifiedName != null
&& !metaItemTypeFullyQualifiedName.equals(itemTypeFullyQualifiedName)) {
- lockMgr.acquireDataTypeReadLock(locks, metaItemTypeFullyQualifiedName);
+ if (metaItemTypeAnonymous) {
+ // the datatype will be created
+ lockMgr.acquireDataTypeWriteLock(locks, metaItemTypeFullyQualifiedName);
+ } else {
+ lockMgr.acquireDataTypeReadLock(locks, metaItemTypeFullyQualifiedName);
+ }
}
if (nodeGroupName != null) {
lockMgr.acquireNodeGroupReadLock(locks, nodeGroupName);
@@ -50,6 +61,16 @@
if (!isDefaultCompactionPolicy) {
lockMgr.acquireMergePolicyReadLock(locks, compactionPolicyName);
}
+ }
+
+ public static void createDatasetBegin(IMetadataLockManager lockMgr, LockList locks, String dataverseName,
+ String itemTypeDataverseName, String itemTypeFullyQualifiedName, boolean itemTypeAnonymous,
+ String metaItemTypeDataverseName, String metaItemTypeFullyQualifiedName, boolean metaItemTypeAnonymous,
+ String nodeGroupName, String compactionPolicyName, String datasetFullyQualifiedName,
+ boolean isDefaultCompactionPolicy) throws AlgebricksException {
+ createDatasetBeginPre(lockMgr, locks, dataverseName, itemTypeDataverseName, itemTypeFullyQualifiedName,
+ itemTypeAnonymous, metaItemTypeDataverseName, metaItemTypeFullyQualifiedName, metaItemTypeAnonymous,
+ nodeGroupName, compactionPolicyName, isDefaultCompactionPolicy);
lockMgr.acquireDatasetWriteLock(locks, datasetFullyQualifiedName);
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataUtil.java
index e5d4721..3436b44 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/MetadataUtil.java
@@ -43,4 +43,5 @@
int idx = datasetName.indexOf('.');
return datasetName.substring(0, idx);
}
+
}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java
index efe795b..4c3ad73 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/TypeUtil.java
@@ -212,4 +212,32 @@
"Field accessor is not defined for \"" + fName + "\" of type " + actualType.getTypeTag());
}
}
+
+ /**
+ * Warning: this is not a general-purpose method.
+ * Use only when processing types stored in metadata.
+ * Doesn't properly handle ANY, UNION, NULL and MISSING types.
+ * Allows {@code null} type reference which will be filled later during type translation.
+ */
+ public static IAType createQuantifiedType(IAType primeType, boolean nullable, boolean missable) {
+ if (primeType != null) {
+ switch (primeType.getTypeTag()) {
+ case ANY:
+ case UNION:
+ case NULL:
+ case MISSING:
+ throw new IllegalArgumentException(primeType.getDisplayName());
+ }
+ }
+
+ IAType resType = primeType;
+ if (nullable && missable) {
+ resType = AUnionType.createUnknownableType(resType);
+ } else if (nullable) {
+ resType = AUnionType.createNullableType(resType);
+ } else if (missable) {
+ resType = AUnionType.createMissableType(resType);
+ }
+ return resType;
+ }
}
diff --git a/asterixdb/asterix-om/pom.xml b/asterixdb/asterix-om/pom.xml
index fc1707f..b123850 100644
--- a/asterixdb/asterix-om/pom.xml
+++ b/asterixdb/asterix-om/pom.xml
@@ -38,6 +38,28 @@
<root.dir>${basedir}/..</root.dir>
</properties>
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-surefire-plugin</artifactId>
+ <configuration>
+ <parallel>all</parallel>
+ <forkCount>4</forkCount>
+ <reuseForks>false</reuseForks>
+ <threadCount>1</threadCount>
+ <argLine>
+ -enableassertions -Xmx${test.heap.size}m
+ -Dfile.encoding=UTF-8
+ -DrunSlowAQLTests=${runSlowAQLTests}
+ -Xdebug
+ ${coverageArgLine}
+ </argLine>
+ </configuration>
+ </plugin>
+ </plugins>
+ </build>
+
<dependencies>
<dependency>
<groupId>org.apache.asterix</groupId>
@@ -61,7 +83,7 @@
</dependency>
<dependency>
<groupId>org.mockito</groupId>
- <artifactId>mockito-all</artifactId>
+ <artifactId>mockito-core</artifactId>
<scope>test</scope>
</dependency>
<dependency>
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APolygonSerializerDeserializer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APolygonSerializerDeserializer.java
index 83810e3..10b61c0 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APolygonSerializerDeserializer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/APolygonSerializerDeserializer.java
@@ -23,6 +23,7 @@
import java.io.IOException;
import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.dataflow.data.nontagged.Coordinate;
import org.apache.asterix.om.base.APoint;
import org.apache.asterix.om.base.APolygon;
@@ -70,7 +71,7 @@
case Y:
return 11 + (pointId * 16);
default:
- throw HyracksDataException.create(ErrorCode.POLYGON_INVALID_COORDINATE);
+ throw new RuntimeDataException(ErrorCode.POLYGON_INVALID_COORDINATE);
}
}
@@ -78,7 +79,7 @@
try {
String[] points = polygon.split(" ");
if (points.length < 3) {
- throw HyracksDataException.create(ErrorCode.POLYGON_3_POINTS);
+ throw new RuntimeDataException(ErrorCode.POLYGON_3_POINTS);
}
out.writeByte(ATypeTag.POLYGON.serialize());
out.writeShort(points.length);
@@ -87,7 +88,7 @@
APointSerializerDeserializer.serialize(Double.parseDouble(split[0]), Double.parseDouble(split[1]), out);
}
} catch (IOException e) {
- throw HyracksDataException.create(ErrorCode.POLYGON_INVALID, e, polygon);
+ throw new RuntimeDataException(ErrorCode.POLYGON_INVALID, e, polygon);
}
}
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/SerializerDeserializerUtil.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/SerializerDeserializerUtil.java
index 952ffb4..664eb28 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/SerializerDeserializerUtil.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/serde/SerializerDeserializerUtil.java
@@ -37,6 +37,10 @@
public final class SerializerDeserializerUtil {
+ private SerializerDeserializerUtil() {
+ throw new AssertionError("do not instantiate");
+ }
+
public static void writeIntToByteArray(byte[] array, int value, int offset) {
array[offset] = (byte) (0xff & (value >> 24));
array[offset + 1] = (byte) (0xff & (value >> 16));
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmObjectNode.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmObjectNode.java
index 26024b4..294f48b 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmObjectNode.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmObjectNode.java
@@ -28,6 +28,7 @@
import org.apache.asterix.builders.RecordBuilder;
import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.om.base.AString;
import org.apache.asterix.om.types.ATypeTag;
@@ -164,9 +165,9 @@
public static String getString(AdmObjectNode openFields, String field) throws HyracksDataException {
IAdmNode node = openFields.get(field);
if (node == null) {
- throw HyracksDataException.create(ErrorCode.FIELD_NOT_FOUND, field);
+ throw new RuntimeDataException(ErrorCode.FIELD_NOT_FOUND, field);
} else if (node.getType() != ATypeTag.STRING) {
- throw HyracksDataException.create(ErrorCode.FIELD_NOT_OF_TYPE, field, ATypeTag.STRING, node.getType());
+ throw new RuntimeDataException(ErrorCode.FIELD_NOT_OF_TYPE, field, ATypeTag.STRING, node.getType());
}
return ((AdmStringNode) node).get();
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ADateParserFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ADateParserFactory.java
index ed118f9..35691a1 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ADateParserFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ADateParserFactory.java
@@ -43,9 +43,10 @@
return new IValueParser() {
@Override
- public void parse(char[] buffer, int start, int length, DataOutput out) throws HyracksDataException {
+ public boolean parse(char[] buffer, int start, int length, DataOutput out) throws HyracksDataException {
try {
out.writeInt((int) (parseDatePart(buffer, start, length) / GregorianCalendarSystem.CHRONON_OF_DAY));
+ return true;
} catch (IOException ex) {
throw HyracksDataException.create(ex);
}
@@ -56,11 +57,11 @@
/**
* Parse the given char sequence as a date string, and return the milliseconds represented by the date.
*
- * @param charAccessor
+ * @param dateString
* accessor for the char sequence
- * @param isDateOnly
+ * @param start
* indicating whether it is a single date string, or it is the date part of a datetime string
- * @param errorMessage
+ * @param length
* @return
* @throws Exception
*/
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ADateTimeParserFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ADateTimeParserFactory.java
index 2e32692..f9e53b9 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ADateTimeParserFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ADateTimeParserFactory.java
@@ -43,7 +43,7 @@
return new IValueParser() {
@Override
- public void parse(char[] buffer, int start, int length, DataOutput out) throws HyracksDataException {
+ public boolean parse(char[] buffer, int start, int length, DataOutput out) throws HyracksDataException {
long chrononTimeInMs = 0;
short timeOffset = (short) ((buffer[start] == '-') ? 1 : 0);
@@ -64,6 +64,7 @@
try {
out.writeLong(chrononTimeInMs);
+ return true;
} catch (IOException ex) {
throw HyracksDataException.create(ex);
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ADurationParserFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ADurationParserFactory.java
index 66fb4c3..ec387e2 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ADurationParserFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ADurationParserFactory.java
@@ -52,11 +52,12 @@
return new IValueParser() {
@Override
- public void parse(char[] buffer, int start, int length, DataOutput out) throws HyracksDataException {
+ public boolean parse(char[] buffer, int start, int length, DataOutput out) throws HyracksDataException {
parseDuration(buffer, start, length, aMutableDuration, ADurationParseOption.All);
try {
out.writeInt(aMutableDuration.getMonths());
out.writeLong(aMutableDuration.getMilliseconds());
+ return true;
} catch (IOException ex) {
throw HyracksDataException.create(ex);
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ATimeParserFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ATimeParserFactory.java
index 40ddc55..039e026 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ATimeParserFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/base/temporal/ATimeParserFactory.java
@@ -43,9 +43,10 @@
return new IValueParser() {
@Override
- public void parse(char[] buffer, int start, int length, DataOutput out) throws HyracksDataException {
+ public boolean parse(char[] buffer, int start, int length, DataOutput out) throws HyracksDataException {
try {
out.writeInt(parseTimePart(buffer, start, length));
+ return true;
} catch (IOException ex) {
throw HyracksDataException.create(ex);
}
@@ -318,7 +319,7 @@
*
* @param timeString
* @param start
- * @param length
+ *
* @return
* @throws HyracksDataException
*/
@@ -473,7 +474,7 @@
*
* @param timeString
* @param start
- * @param length
+ *
* @return
* @throws HyracksDataException
*/
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/ExceptionUtil.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/ExceptionUtil.java
index b5d599b..c9686ee 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/ExceptionUtil.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/ExceptionUtil.java
@@ -136,7 +136,7 @@
}
private static void warnInvalidValue(IEvaluatorContext ctx, SourceLocation srcLoc, FunctionIdentifier fid,
- int argIdx, double argValue, int errorCode) {
+ int argIdx, double argValue, ErrorCode errorCode) {
IWarningCollector warningCollector = ctx.getWarningCollector();
if (warningCollector.shouldWarn()) {
warningCollector.warn(WarningUtil.forAsterix(srcLoc, errorCode, fid.getName(), indexToPosition(argIdx),
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/TypeMismatchException.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/TypeMismatchException.java
index e155b94..3b8602b 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/TypeMismatchException.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/exceptions/TypeMismatchException.java
@@ -32,14 +32,13 @@
private static final long serialVersionUID = -3069967719104299912L;
// Function parameter type mismatch.
- public TypeMismatchException(FunctionIdentifier fid, Integer i, ATypeTag actualTypeTag,
- ATypeTag... expectedTypeTags) {
+ public TypeMismatchException(FunctionIdentifier fid, int i, ATypeTag actualTypeTag, ATypeTag... expectedTypeTags) {
super(ErrorCode.COMPILATION_TYPE_MISMATCH_FUNCTION, fid.getName(), indexToPosition(i),
toExpectedTypeString(expectedTypeTags), actualTypeTag);
}
// Function parameter type mismatch.
- public TypeMismatchException(SourceLocation sourceLoc, FunctionIdentifier fid, Integer i, ATypeTag actualTypeTag,
+ public TypeMismatchException(SourceLocation sourceLoc, FunctionIdentifier fid, int i, ATypeTag actualTypeTag,
ATypeTag... expectedTypeTags) {
super(ErrorCode.COMPILATION_TYPE_MISMATCH_FUNCTION, sourceLoc, fid.getName(), indexToPosition(i),
toExpectedTypeString(expectedTypeTags), actualTypeTag);
@@ -47,14 +46,14 @@
// Function parameter type mismatch.
@Deprecated
- public TypeMismatchException(String functionName, Integer i, ATypeTag actualTypeTag, ATypeTag... expectedTypeTags) {
+ public TypeMismatchException(String functionName, int i, ATypeTag actualTypeTag, ATypeTag... expectedTypeTags) {
super(ErrorCode.COMPILATION_TYPE_MISMATCH_FUNCTION, functionName, indexToPosition(i),
toExpectedTypeString(expectedTypeTags), actualTypeTag);
}
// Function parameter type mismatch.
@Deprecated
- public TypeMismatchException(SourceLocation sourceLoc, String functionName, Integer i, ATypeTag actualTypeTag,
+ public TypeMismatchException(SourceLocation sourceLoc, String functionName, int i, ATypeTag actualTypeTag,
ATypeTag... expectedTypeTags) {
super(ErrorCode.COMPILATION_TYPE_MISMATCH_FUNCTION, sourceLoc, functionName, indexToPosition(i),
toExpectedTypeString(expectedTypeTags), actualTypeTag);
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
index bf2709a..f2a1edf 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
@@ -1771,6 +1771,11 @@
addFunction(NEGINF_IF, DoubleIfTypeComputer.INSTANCE, true);
// Aggregate Functions
+ ScalarVersionOfAggregateResultType scalarNumericSumTypeComputer =
+ new ScalarVersionOfAggregateResultType(NumericSumAggTypeComputer.INSTANCE);
+ ScalarVersionOfAggregateResultType scalarMinMaxTypeComputer =
+ new ScalarVersionOfAggregateResultType(MinMaxAggTypeComputer.INSTANCE);
+
addPrivateFunction(LISTIFY, OrderedListConstructorTypeComputer.INSTANCE, true);
addFunction(SCALAR_ARRAYAGG, ScalarArrayAggTypeComputer.INSTANCE, true);
addFunction(MAX, MinMaxAggTypeComputer.INSTANCE, true);
@@ -1816,7 +1821,7 @@
// SUM
addFunction(SUM, NumericSumAggTypeComputer.INSTANCE, true);
- addFunction(SCALAR_SUM, ScalarVersionOfAggregateResultType.INSTANCE, true);
+ addFunction(SCALAR_SUM, scalarNumericSumTypeComputer, true);
addPrivateFunction(LOCAL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
addPrivateFunction(INTERMEDIATE_SUM, NumericSumAggTypeComputer.INSTANCE, true);
addPrivateFunction(GLOBAL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
@@ -1832,8 +1837,8 @@
addPrivateFunction(SERIAL_INTERMEDIATE_SQL_AVG, LocalAvgTypeComputer.INSTANCE, true);
addFunction(SCALAR_AVG, NullableDoubleTypeComputer.INSTANCE, true);
addFunction(SCALAR_COUNT, AInt64TypeComputer.INSTANCE, true);
- addFunction(SCALAR_MAX, ScalarVersionOfAggregateResultType.INSTANCE, true);
- addFunction(SCALAR_MIN, ScalarVersionOfAggregateResultType.INSTANCE, true);
+ addFunction(SCALAR_MAX, scalarMinMaxTypeComputer, true);
+ addFunction(SCALAR_MIN, scalarMinMaxTypeComputer, true);
addPrivateFunction(INTERMEDIATE_AVG, LocalAvgTypeComputer.INSTANCE, true);
addFunction(SCALAR_STDDEV_SAMP, NullableDoubleTypeComputer.INSTANCE, true);
addPrivateFunction(INTERMEDIATE_STDDEV_SAMP, LocalSingleVarStatisticsTypeComputer.INSTANCE, true);
@@ -1874,7 +1879,7 @@
// SQL SUM
addFunction(SQL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
- addFunction(SCALAR_SQL_SUM, ScalarVersionOfAggregateResultType.INSTANCE, true);
+ addFunction(SCALAR_SQL_SUM, scalarNumericSumTypeComputer, true);
addPrivateFunction(LOCAL_SQL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
addPrivateFunction(INTERMEDIATE_SQL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
addPrivateFunction(GLOBAL_SQL_SUM, NumericSumAggTypeComputer.INSTANCE, true);
@@ -1898,8 +1903,8 @@
addPrivateFunction(GLOBAL_SQL_MIN, MinMaxAggTypeComputer.INSTANCE, true);
addFunction(SCALAR_SQL_AVG, NullableDoubleTypeComputer.INSTANCE, true);
addFunction(SCALAR_SQL_COUNT, AInt64TypeComputer.INSTANCE, true);
- addFunction(SCALAR_SQL_MAX, ScalarVersionOfAggregateResultType.INSTANCE, true);
- addFunction(SCALAR_SQL_MIN, ScalarVersionOfAggregateResultType.INSTANCE, true);
+ addFunction(SCALAR_SQL_MAX, scalarMinMaxTypeComputer, true);
+ addFunction(SCALAR_SQL_MIN, scalarMinMaxTypeComputer, true);
addPrivateFunction(INTERMEDIATE_SQL_AVG, LocalAvgTypeComputer.INSTANCE, true);
addFunction(SQL_STDDEV_SAMP, NullableDoubleTypeComputer.INSTANCE, true);
addPrivateFunction(GLOBAL_SQL_STDDEV_SAMP, NullableDoubleTypeComputer.INSTANCE, true);
@@ -1974,9 +1979,9 @@
addFunction(SCALAR_SQL_COUNT_DISTINCT, AInt64TypeComputer.INSTANCE, true);
addFunction(SUM_DISTINCT, NumericSumAggTypeComputer.INSTANCE, true);
- addFunction(SCALAR_SUM_DISTINCT, ScalarVersionOfAggregateResultType.INSTANCE, true);
+ addFunction(SCALAR_SUM_DISTINCT, scalarNumericSumTypeComputer, true);
addFunction(SQL_SUM_DISTINCT, NumericSumAggTypeComputer.INSTANCE, true);
- addFunction(SCALAR_SQL_SUM_DISTINCT, ScalarVersionOfAggregateResultType.INSTANCE, true);
+ addFunction(SCALAR_SQL_SUM_DISTINCT, scalarNumericSumTypeComputer, true);
addFunction(AVG_DISTINCT, NullableDoubleTypeComputer.INSTANCE, true);
addFunction(SCALAR_AVG_DISTINCT, NullableDoubleTypeComputer.INSTANCE, true);
@@ -1984,14 +1989,14 @@
addFunction(SCALAR_SQL_AVG_DISTINCT, NullableDoubleTypeComputer.INSTANCE, true);
addFunction(MAX_DISTINCT, MinMaxAggTypeComputer.INSTANCE, true);
- addFunction(SCALAR_MAX_DISTINCT, ScalarVersionOfAggregateResultType.INSTANCE, true);
+ addFunction(SCALAR_MAX_DISTINCT, scalarMinMaxTypeComputer, true);
addFunction(SQL_MAX_DISTINCT, MinMaxAggTypeComputer.INSTANCE, true);
- addFunction(SCALAR_SQL_MAX_DISTINCT, ScalarVersionOfAggregateResultType.INSTANCE, true);
+ addFunction(SCALAR_SQL_MAX_DISTINCT, scalarMinMaxTypeComputer, true);
addFunction(MIN_DISTINCT, MinMaxAggTypeComputer.INSTANCE, true);
- addFunction(SCALAR_MIN_DISTINCT, ScalarVersionOfAggregateResultType.INSTANCE, true);
+ addFunction(SCALAR_MIN_DISTINCT, scalarMinMaxTypeComputer, true);
addFunction(SQL_MIN_DISTINCT, MinMaxAggTypeComputer.INSTANCE, true);
- addFunction(SCALAR_SQL_MIN_DISTINCT, ScalarVersionOfAggregateResultType.INSTANCE, true);
+ addFunction(SCALAR_SQL_MIN_DISTINCT, scalarMinMaxTypeComputer, true);
addFunction(STDDEV_SAMP_DISTINCT, NullableDoubleTypeComputer.INSTANCE, true);
addFunction(SCALAR_STDDEV_SAMP_DISTINCT, NullableDoubleTypeComputer.INSTANCE, true);
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/ARecordPointable.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/ARecordPointable.java
index af8d5e6..cc3816c 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/ARecordPointable.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/pointables/nonvisitor/ARecordPointable.java
@@ -152,7 +152,7 @@
&& RecordUtil.isNull(bytes[getNullBitmapOffset(recordType) + fieldId / 4], fieldId);
}
- private boolean isClosedFieldMissing(ARecordType recordType, int fieldId) {
+ public boolean isClosedFieldMissing(ARecordType recordType, int fieldId) {
return getNullBitmapSize(recordType) > 0
&& RecordUtil.isMissing(bytes[getNullBitmapOffset(recordType) + fieldId / 4], fieldId);
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AggregateResultTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AggregateResultTypeComputer.java
new file mode 100644
index 0000000..8e663a7
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AggregateResultTypeComputer.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+
+public abstract class AggregateResultTypeComputer extends AbstractResultTypeComputer {
+ @Override
+ protected void checkArgType(FunctionIdentifier funcId, int argIndex, IAType type, SourceLocation sourceLoc)
+ throws AlgebricksException {
+ super.checkArgType(funcId, argIndex, type, sourceLoc);
+ }
+
+ @Override
+ protected abstract IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes)
+ throws AlgebricksException;
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MinMaxAggTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MinMaxAggTypeComputer.java
index c34b5ed..fc1eee5 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MinMaxAggTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/MinMaxAggTypeComputer.java
@@ -19,7 +19,6 @@
package org.apache.asterix.om.typecomputer.impl;
import org.apache.asterix.dataflow.data.common.ILogicalBinaryComparator;
-import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AUnionType;
import org.apache.asterix.om.types.BuiltinType;
@@ -27,7 +26,7 @@
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-public class MinMaxAggTypeComputer extends AbstractResultTypeComputer {
+public class MinMaxAggTypeComputer extends AggregateResultTypeComputer {
public static final MinMaxAggTypeComputer INSTANCE = new MinMaxAggTypeComputer();
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericSumAggTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericSumAggTypeComputer.java
index 1c67e56..a4b5e34 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericSumAggTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/NumericSumAggTypeComputer.java
@@ -18,42 +18,20 @@
*/
package org.apache.asterix.om.typecomputer.impl;
-import org.apache.asterix.om.exceptions.UnsupportedTypeException;
-import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AUnionType;
import org.apache.asterix.om.types.BuiltinType;
import org.apache.asterix.om.types.IAType;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import org.apache.hyracks.api.exceptions.SourceLocation;
-public class NumericSumAggTypeComputer extends AbstractResultTypeComputer {
+public class NumericSumAggTypeComputer extends AggregateResultTypeComputer {
public static final NumericSumAggTypeComputer INSTANCE = new NumericSumAggTypeComputer();
private NumericSumAggTypeComputer() {
}
@Override
- protected void checkArgType(FunctionIdentifier funcId, int argIndex, IAType type, SourceLocation sourceLoc)
- throws AlgebricksException {
- ATypeTag tag = type.getTypeTag();
- switch (tag) {
- case DOUBLE:
- case FLOAT:
- case BIGINT:
- case INTEGER:
- case SMALLINT:
- case TINYINT:
- case ANY:
- break;
- default:
- throw new UnsupportedTypeException(sourceLoc, funcId, tag);
- }
- }
-
- @Override
protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
ATypeTag tag = strippedInputTypes[0].getTypeTag();
switch (tag) {
@@ -61,15 +39,12 @@
case SMALLINT:
case INTEGER:
case BIGINT:
- IAType int64Type = BuiltinType.AINT64;
- return AUnionType.createNullableType(int64Type, "AggResult");
+ return AUnionType.createNullableType(BuiltinType.AINT64);
case FLOAT:
case DOUBLE:
- IAType doubleType = BuiltinType.ADOUBLE;
- return AUnionType.createNullableType(doubleType, "AggResult");
+ return AUnionType.createNullableType(BuiltinType.ADOUBLE);
case ANY:
- IAType anyType = strippedInputTypes[0];
- return AUnionType.createNullableType(anyType, "AggResult");
+ return BuiltinType.ANY;
default:
// All other possible cases.
return BuiltinType.ANULL;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ScalarVersionOfAggregateResultType.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ScalarVersionOfAggregateResultType.java
index 5b90974..fda0285 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ScalarVersionOfAggregateResultType.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ScalarVersionOfAggregateResultType.java
@@ -18,9 +18,7 @@
*/
package org.apache.asterix.om.typecomputer.impl;
-import org.apache.asterix.om.exceptions.TypeMismatchException;
import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
-import org.apache.asterix.om.types.ATypeTag;
import org.apache.asterix.om.types.AUnionType;
import org.apache.asterix.om.types.AbstractCollectionType;
import org.apache.asterix.om.types.BuiltinType;
@@ -32,32 +30,48 @@
public class ScalarVersionOfAggregateResultType extends AbstractResultTypeComputer {
- public static final ScalarVersionOfAggregateResultType INSTANCE = new ScalarVersionOfAggregateResultType();
+ private final AggregateResultTypeComputer aggResultTypeComputer;
- private ScalarVersionOfAggregateResultType() {
+ public ScalarVersionOfAggregateResultType(AggregateResultTypeComputer aggResultTypeComputer) {
+ this.aggResultTypeComputer = aggResultTypeComputer;
}
@Override
- public void checkArgType(FunctionIdentifier funcId, int argIndex, IAType type, SourceLocation sourceLoc)
+ protected void checkArgType(FunctionIdentifier funcId, int argIndex, IAType argType, SourceLocation sourceLoc)
throws AlgebricksException {
- ATypeTag tag = type.getTypeTag();
- if (tag != ATypeTag.ANY && tag != ATypeTag.ARRAY && tag != ATypeTag.MULTISET) {
- throw new TypeMismatchException(sourceLoc, funcId, argIndex, tag, ATypeTag.ARRAY, ATypeTag.MULTISET);
+ if (argIndex == 0) {
+ switch (argType.getTypeTag()) {
+ case ARRAY:
+ case MULTISET:
+ AbstractCollectionType act = (AbstractCollectionType) argType;
+ aggResultTypeComputer.checkArgType(funcId, argIndex, act.getItemType(), sourceLoc);
+ break;
+ }
}
}
@Override
protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
- ATypeTag tag = strippedInputTypes[0].getTypeTag();
- if (tag == ATypeTag.ANY) {
- return BuiltinType.ANY;
+ IAType argType = strippedInputTypes[0];
+ switch (argType.getTypeTag()) {
+ case ARRAY:
+ case MULTISET:
+ AbstractCollectionType act = (AbstractCollectionType) argType;
+ IAType[] strippedInputTypes2 = strippedInputTypes.clone();
+ strippedInputTypes2[0] = act.getItemType();
+ IAType resultType = aggResultTypeComputer.getResultType(expr, strippedInputTypes2);
+ switch (resultType.getTypeTag()) {
+ case NULL:
+ case MISSING:
+ case ANY:
+ return resultType;
+ case UNION:
+ return AUnionType.createUnknownableType(((AUnionType) resultType).getActualType());
+ default:
+ return AUnionType.createUnknownableType(resultType);
+ }
+ default:
+ return BuiltinType.ANY;
}
- if (tag != ATypeTag.ARRAY && tag != ATypeTag.MULTISET) {
- // this condition being true would've thrown an exception above, no?
- return strippedInputTypes[0];
- }
- AbstractCollectionType act = (AbstractCollectionType) strippedInputTypes[0];
- IAType t = act.getItemType();
- return AUnionType.createUnknownableType(t);
}
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/NonTaggedFormatUtil.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/NonTaggedFormatUtil.java
index 7da0263..4a1d371 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/NonTaggedFormatUtil.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/utils/NonTaggedFormatUtil.java
@@ -20,6 +20,7 @@
import org.apache.asterix.common.config.DatasetConfig.IndexType;
import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
import org.apache.asterix.dataflow.data.nontagged.serde.AIntervalSerializerDeserializer;
@@ -114,7 +115,7 @@
case ANY:
ATypeTag tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serNonTaggedAObject[offset]);
if (tag == ATypeTag.ANY) {
- throw HyracksDataException.create(ErrorCode.FIELD_SHOULD_BE_TYPED);
+ throw new RuntimeDataException(ErrorCode.FIELD_SHOULD_BE_TYPED);
}
return getFieldValueLength(serNonTaggedAObject, offset, tag, true) + 1;
case MISSING:
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/PartitionReplica.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/PartitionReplica.java
index 3226299..6b97306 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/PartitionReplica.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/api/PartitionReplica.java
@@ -113,10 +113,10 @@
public synchronized void close() {
try {
- if (sc != null) {
+ if (NetworkingUtil.isHealthy(sc)) {
sendGoodBye();
- NetworkUtil.closeQuietly(sc);
}
+ NetworkUtil.closeQuietly(sc);
} finally {
sc = null;
}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/IndexReplicationManager.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/IndexReplicationManager.java
index 4d660d2..dd953c4 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/IndexReplicationManager.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/IndexReplicationManager.java
@@ -131,9 +131,14 @@
private boolean skip(IReplicationJob job) {
try {
- final DatasetResourceReference indexFileRef =
- resourceRepository.getLocalResourceReference(job.getAnyFile());
- return !replicationStrategy.isMatch(indexFileRef.getDatasetId());
+ final String fileToReplicate = job.getAnyFile();
+ final Optional<DatasetResourceReference> indexFileRefOpt =
+ resourceRepository.getLocalResourceReference(fileToReplicate);
+ if (!indexFileRefOpt.isPresent()) {
+ LOGGER.warn("skipping replication of {} due to missing dataset resource reference", fileToReplicate);
+ return true;
+ }
+ return !replicationStrategy.isMatch(indexFileRefOpt.get().getDatasetId());
} catch (HyracksDataException e) {
throw new IllegalStateException("Couldn't find resource for " + job.getAnyFile(), e);
}
diff --git a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/NetworkingUtil.java b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/NetworkingUtil.java
index d572ba2..7f6439c 100644
--- a/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/NetworkingUtil.java
+++ b/asterixdb/asterix-replication/src/main/java/org/apache/asterix/replication/management/NetworkingUtil.java
@@ -34,9 +34,13 @@
import org.apache.hyracks.api.comm.NetworkAddress;
import org.apache.hyracks.api.network.ISocketChannel;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
public class NetworkingUtil {
+ private static final Logger LOGGER = LogManager.getLogger();
+
private NetworkingUtil() {
throw new AssertionError("This util class should not be initialized.");
}
@@ -56,25 +60,33 @@
public static void sendFile(FileChannel fileChannel, ISocketChannel socketChannel) throws IOException {
long pos = 0;
- long fileSize = fileChannel.size();
- long remainingBytes = fileSize;
- long transferredBytes = 0;
-
- while ((transferredBytes += fileChannel.transferTo(pos, remainingBytes, socketChannel)) < fileSize) {
- pos += transferredBytes;
- remainingBytes -= transferredBytes;
+ long remainingBytes = fileChannel.size();
+ try {
+ while (remainingBytes > 0) {
+ long sentBytes = fileChannel.transferTo(pos, remainingBytes, socketChannel);
+ pos += sentBytes;
+ remainingBytes -= sentBytes;
+ }
+ socketChannel.getSocketChannel().socket().getOutputStream().flush();
+ } catch (Exception e) {
+ LOGGER.info("failed to send file; file size {}, pos {}, remainingBytes {}", fileChannel.size(), pos,
+ remainingBytes);
}
- socketChannel.getSocketChannel().socket().getOutputStream().flush();
}
public static void downloadFile(FileChannel fileChannel, ISocketChannel socketChannel) throws IOException {
+ long remainingBytes = fileChannel.size();
long pos = 0;
- long fileSize = fileChannel.size();
- long count = fileSize;
- long numTransferred = 0;
- while ((numTransferred += fileChannel.transferFrom(socketChannel, pos, count)) < fileSize) {
- pos += numTransferred;
- count -= numTransferred;
+ try {
+ while (remainingBytes > 0) {
+ long readBytes = fileChannel.transferFrom(socketChannel, pos, remainingBytes);
+ pos += readBytes;
+ remainingBytes -= readBytes;
+ }
+ } catch (Exception e) {
+ LOGGER.info("failed to download file; file size {}, pos {}, remainingBytes {}", fileChannel.size(), pos,
+ remainingBytes);
+ throw e;
}
}
diff --git a/asterixdb/asterix-runtime/pom.xml b/asterixdb/asterix-runtime/pom.xml
index f3d5e9c..b95d90c 100644
--- a/asterixdb/asterix-runtime/pom.xml
+++ b/asterixdb/asterix-runtime/pom.xml
@@ -139,7 +139,7 @@
</dependency>
<dependency>
<groupId>org.mockito</groupId>
- <artifactId>mockito-all</artifactId>
+ <artifactId>mockito-core</artifactId>
<scope>test</scope>
</dependency>
<dependency>
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java
index f77e6c4..bf8698d 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/serializable/std/AbstractSerializableAvgAggregateFunction.java
@@ -35,6 +35,7 @@
import org.apache.asterix.om.base.AMutableDouble;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.om.exceptions.ExceptionUtil;
import org.apache.asterix.om.functions.BuiltinFunctions;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
@@ -44,8 +45,8 @@
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
import org.apache.asterix.runtime.evaluators.common.AccessibleByteArrayEval;
import org.apache.asterix.runtime.evaluators.common.ClosedRecordConstructorEvalFactory.ClosedRecordConstructorEval;
-import org.apache.asterix.runtime.exceptions.IncompatibleTypeException;
import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -65,6 +66,11 @@
private static final int COUNT_OFFSET = 8;
protected static final int AGG_TYPE_OFFSET = 16;
+ private final IEvaluatorContext context;
+
+ // Warning flag to warn only once in case of non-numeric data
+ private boolean isWarned;
+
private IPointable inputVal = new VoidPointable();
private IScalarEvaluator eval;
private AMutableDouble aDouble = new AMutableDouble(0);
@@ -92,6 +98,7 @@
public AbstractSerializableAvgAggregateFunction(IScalarEvaluatorFactory[] args, IEvaluatorContext context,
SourceLocation sourceLoc) throws HyracksDataException {
super(sourceLoc);
+ this.context = context;
eval = args[0].createScalarEvaluator(context);
}
@@ -101,6 +108,7 @@
state.writeDouble(0.0);
state.writeLong(0);
state.writeByte(ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+ isWarned = false;
} catch (IOException e) {
throw HyracksDataException.create(e);
}
@@ -136,17 +144,16 @@
} else if (aggType == ATypeTag.SYSTEM_NULL) {
aggType = typeTag;
} else if (typeTag != ATypeTag.SYSTEM_NULL && !ATypeHierarchy.isCompatible(typeTag, aggType)) {
- if (typeTag.ordinal() > aggType.ordinal()) {
- throw new IncompatibleTypeException(sourceLoc, BuiltinFunctions.AVG, bytes[offset],
- aggType.serialize());
- } else {
- throw new IncompatibleTypeException(sourceLoc, BuiltinFunctions.AVG, aggType.serialize(),
- bytes[offset]);
+ // Issue warning only once and treat current tuple as null
+ if (!isWarned) {
+ isWarned = true;
+ ExceptionUtil.warnUnsupportedType(context, sourceLoc, getIdentifier().getName(), typeTag);
}
+ processNull(state, start);
+ return;
} else if (ATypeHierarchy.canPromote(aggType, typeTag)) {
aggType = typeTag;
}
- ++count;
switch (typeTag) {
case TINYINT: {
byte val = AInt8SerializerDeserializer.getByte(bytes, offset + 1);
@@ -179,8 +186,15 @@
break;
}
default:
- throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.AVG, bytes[offset]);
+ // Issue warning only once and treat current tuple as null
+ if (!isWarned) {
+ isWarned = true;
+ ExceptionUtil.warnUnsupportedType(context, sourceLoc, getIdentifier().getName(), typeTag);
+ }
+ processNull(state, start);
+ return;
}
+ count++;
BufferSerDeUtil.writeDouble(sum, state, start + SUM_OFFSET);
BufferSerDeUtil.writeLong(count, state, start + COUNT_OFFSET);
state[start + AGG_TYPE_OFFSET] = aggType.serialize();
@@ -279,4 +293,8 @@
return false;
}
+ // Function identifier
+ private FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.AVG;
+ }
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java
index 29719ad..d2367f9 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/AbstractAvgAggregateFunction.java
@@ -35,6 +35,7 @@
import org.apache.asterix.om.base.AMutableDouble;
import org.apache.asterix.om.base.AMutableInt64;
import org.apache.asterix.om.base.ANull;
+import org.apache.asterix.om.exceptions.ExceptionUtil;
import org.apache.asterix.om.functions.BuiltinFunctions;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
@@ -44,8 +45,8 @@
import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
import org.apache.asterix.runtime.evaluators.common.AccessibleByteArrayEval;
import org.apache.asterix.runtime.evaluators.common.ClosedRecordConstructorEvalFactory.ClosedRecordConstructorEval;
-import org.apache.asterix.runtime.exceptions.IncompatibleTypeException;
import org.apache.asterix.runtime.exceptions.UnsupportedItemTypeException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
@@ -62,6 +63,11 @@
private static final int SUM_FIELD_ID = 0;
private static final int COUNT_FIELD_ID = 1;
+ private final IEvaluatorContext context;
+
+ // Warning flag to warn only once in case of non-numeric data
+ private boolean isWarned;
+
private final ARecordType recType;
private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
@@ -95,6 +101,7 @@
public AbstractAvgAggregateFunction(IScalarEvaluatorFactory[] args, IEvaluatorContext context,
SourceLocation sourceLoc) throws HyracksDataException {
super(sourceLoc);
+ this.context = context;
eval = args[0].createScalarEvaluator(context);
recType = new ARecordType(null, new String[] { "sum", "count" },
new IAType[] { BuiltinType.ADOUBLE, BuiltinType.AINT64 }, false);
@@ -106,6 +113,7 @@
aggType = ATypeTag.SYSTEM_NULL;
sum = 0.0;
count = 0;
+ isWarned = false;
}
@Override
@@ -134,15 +142,17 @@
} else if (aggType == ATypeTag.SYSTEM_NULL) {
aggType = typeTag;
} else if (typeTag != ATypeTag.SYSTEM_NULL && !ATypeHierarchy.isCompatible(typeTag, aggType)) {
- if (typeTag.ordinal() > aggType.ordinal()) {
- throw new IncompatibleTypeException(sourceLoc, BuiltinFunctions.AVG, data[offset], aggType.serialize());
- } else {
- throw new IncompatibleTypeException(sourceLoc, BuiltinFunctions.AVG, aggType.serialize(), data[offset]);
+ // Issue warning only once and treat current tuple as null
+ if (!isWarned) {
+ isWarned = true;
+ ExceptionUtil.warnUnsupportedType(context, sourceLoc, getIdentifier().getName(), typeTag);
}
+ processNull();
+ return;
} else if (ATypeHierarchy.canPromote(aggType, typeTag)) {
aggType = typeTag;
}
- ++count;
+
switch (typeTag) {
case TINYINT: {
byte val = AInt8SerializerDeserializer.getByte(data, offset + 1);
@@ -175,9 +185,16 @@
break;
}
default: {
- throw new UnsupportedItemTypeException(sourceLoc, BuiltinFunctions.AVG, data[offset]);
+ // Issue warning only once and treat current tuple as null
+ if (!isWarned) {
+ isWarned = true;
+ ExceptionUtil.warnUnsupportedType(context, sourceLoc, getIdentifier().getName(), typeTag);
+ }
+ processNull();
+ return;
}
}
+ count++;
}
protected void finishPartialResults(IPointable result) throws HyracksDataException {
@@ -259,4 +276,8 @@
return false;
}
+ // Function identifier
+ private FunctionIdentifier getIdentifier() {
+ return BuiltinFunctions.AVG;
+ }
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryHexStringConstructorDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryHexStringConstructorDescriptor.java
index 3d65b0b..505b650 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryHexStringConstructorDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/constructors/ABinaryHexStringConstructorDescriptor.java
@@ -111,7 +111,10 @@
utf8Ptr.set(inputArg.getByteArray(), startOffset + 1, len - 1);
char[] buffer = utf8Ptr.toString().toCharArray();
out.write(ATypeTag.BINARY.serialize());
- byteArrayParser.parse(buffer, 0, buffer.length, out);
+ if (!byteArrayParser.parse(buffer, 0, buffer.length, out)) {
+ PointableHelper.setNull(result);
+ return;
+ }
result.set(resultStorage);
} else {
throw new TypeMismatchException(sourceLoc, BuiltinFunctions.BINARY_HEX_CONSTRUCTOR, 0, tt,
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java
index f7bddf1..fb71eae 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/GetItemDescriptor.java
@@ -116,6 +116,10 @@
int itemIndex = ATypeHierarchy.getIntegerValue(BuiltinFunctions.GET_ITEM.getName(), 0,
indexBytes, indexOffset);
+ if (itemIndex < 0) {
+ itemIndex = itemCount + itemIndex;
+ }
+
if (itemIndex < 0 || itemIndex >= itemCount) {
// Out-of-bound index access should return MISSING.
result.set(missingBytes, 0, 1);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomWithSeedDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomWithSeedDescriptor.java
index aa49010..bd74f24 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomWithSeedDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/RandomWithSeedDescriptor.java
@@ -24,6 +24,7 @@
import org.apache.asterix.om.functions.IFunctionDescriptor;
import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
import org.apache.asterix.runtime.evaluators.functions.utils.RandomHelper;
import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -78,7 +79,9 @@
case BIGINT:
case FLOAT:
case DOUBLE:
- randomHelper.setSeed(bytes, offset + 1, arg0.getLength() - 1);
+ double seed =
+ ATypeHierarchy.getDoubleValue(getIdentifier().getName(), 0, bytes, offset);
+ randomHelper.setSeed(seed);
randomHelper.nextDouble(resultPointable);
break;
default:
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEvaluatorUtils.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEvaluatorUtils.java
index 81ce832..492d64c 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEvaluatorUtils.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/StringEvaluatorUtils.java
@@ -60,8 +60,8 @@
return destString;
}
- static final char[] RESERVED_REGEX_CHARS = new char[] { '\\', '(', ')', '[', ']', '{', '}', '.', '^', '$', '*', '|',
- '+', '?', '<', '>', '-', '=', '!' };
+ public static final char[] RESERVED_REGEX_CHARS = new char[] { '\\', '(', ')', '[', ']', '{', '}', '.', '^', '$',
+ '*', '|', '+', '?', '<', '>', '-', '=', '!' };
static {
Arrays.sort(RESERVED_REGEX_CHARS);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
index 103a164..d6c0e21 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/GetRecordFieldValueEvalFactory.java
@@ -27,7 +27,9 @@
import org.apache.asterix.om.functions.BuiltinFunctions;
import org.apache.asterix.om.types.ARecordType;
import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.types.IAType;
import org.apache.asterix.om.types.runtime.RuntimeRecordTypeInfo;
import org.apache.asterix.om.utils.NonTaggedFormatUtil;
import org.apache.asterix.om.utils.RecordUtil;
@@ -93,7 +95,6 @@
if (PointableHelper.checkAndSetMissingOrNull(result, inputArg0, inputArg1)) {
return;
}
-
byte[] serFldName = inputArg1.getByteArray();
int serFldNameOffset = inputArg1.getStartOffset();
int serFldNameLen = inputArg1.getLength();
@@ -114,22 +115,32 @@
int subFieldIndex = recTypeInfo.getFieldIndex(serFldName, serFldNameOffset + 1, serFldNameLen - 1);
if (subFieldIndex >= 0) {
int nullBitmapSize = RecordUtil.computeNullBitmapSize(recordType);
+
subFieldOffset = ARecordSerializerDeserializer.getFieldOffsetById(serRecord, serRecordOffset,
subFieldIndex, nullBitmapSize, recordType.isOpen());
- if (subFieldOffset == 0) {
- // the field is null, we checked the null bit map
- out.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
- result.set(resultStorage);
- return;
+ switch (subFieldOffset) {
+ case -1:
+ out.writeByte(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
+ result.set(resultStorage);
+ return;
+ case 0:
+ out.writeByte(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+ result.set(resultStorage);
+ return;
+ default:
+ IAType fieldType = recordType.getFieldTypes()[subFieldIndex];
+ if (fieldType.getTypeTag() == ATypeTag.UNION) {
+ fieldType = ((AUnionType) fieldType).getActualType();
+ }
+ ATypeTag fieldTypeTag = fieldType.getTypeTag();
+ subFieldLength = NonTaggedFormatUtil.getFieldValueLength(serRecord, subFieldOffset,
+ fieldTypeTag, false);
+ // write result.
+ out.writeByte(fieldTypeTag.serialize());
+ out.write(serRecord, subFieldOffset, subFieldLength);
+ result.set(resultStorage);
+ return;
}
- ATypeTag fieldTypeTag = recordType.getFieldTypes()[subFieldIndex].getTypeTag();
- subFieldLength =
- NonTaggedFormatUtil.getFieldValueLength(serRecord, subFieldOffset, fieldTypeTag, false);
- // write result.
- out.writeByte(fieldTypeTag.serialize());
- out.write(serRecord, subFieldOffset, subFieldLength);
- result.set(resultStorage);
- return;
}
// Look at open fields.
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java
index 0d85807..25dd73e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordFieldsUtil.java
@@ -58,16 +58,15 @@
private final static AString nestedName = new AString("nested");
private final static AString listName = new AString("list");
- private IObjectPool<IARecordBuilder, ATypeTag> recordBuilderPool =
- new ListObjectPool<IARecordBuilder, ATypeTag>(new RecordBuilderFactory());
- private IObjectPool<IAsterixListBuilder, ATypeTag> listBuilderPool =
- new ListObjectPool<IAsterixListBuilder, ATypeTag>(new ListBuilderFactory());
- private IObjectPool<IMutableValueStorage, ATypeTag> abvsBuilderPool =
- new ListObjectPool<IMutableValueStorage, ATypeTag>(new AbvsBuilderFactory());
- private IObjectPool<IPointable, ATypeTag> recordPointablePool =
- new ListObjectPool<IPointable, ATypeTag>(ARecordPointable.ALLOCATOR);
- private IObjectPool<IPointable, ATypeTag> listPointablePool =
- new ListObjectPool<IPointable, ATypeTag>(AListPointable.ALLOCATOR);
+ private final IObjectPool<IARecordBuilder, ATypeTag> recordBuilderPool =
+ new ListObjectPool<>(new RecordBuilderFactory());
+ private final IObjectPool<IAsterixListBuilder, ATypeTag> listBuilderPool =
+ new ListObjectPool<>(new ListBuilderFactory());
+ private final IObjectPool<IMutableValueStorage, ATypeTag> abvsBuilderPool =
+ new ListObjectPool<>(new AbvsBuilderFactory());
+ private final IObjectPool<IPointable, ATypeTag> recordPointablePool =
+ new ListObjectPool<>(ARecordPointable.ALLOCATOR);
+ private final IObjectPool<IPointable, ATypeTag> listPointablePool = new ListObjectPool<>(AListPointable.ALLOCATOR);
private final static AOrderedListType listType = new AOrderedListType(BuiltinType.ANY, "fields");
//Better not be a static object.
@@ -114,15 +113,14 @@
// write nested or list types
if (tag == ATypeTag.SERIALIZED_RECORD_TYPE_TAG || tag == ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
|| tag == ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
- if (!recordAccessor.isClosedFieldNull(recType, i)) {
+ if (!recordAccessor.isClosedFieldNull(recType, i) && !recordAccessor.isClosedFieldMissing(recType, i)) {
IAType fieldType = recordAccessor.getClosedFieldType(recType, i);
ArrayBackedValueStorage tmpValue = getTempBuffer();
tmpValue.reset();
recordAccessor.getClosedFieldValue(recType, i, tmpValue.getDataOutput());
if (tag == ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
addNestedField(tmpValue, fieldType, fieldRecordBuilder, level + 1);
- } else if (tag == ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
- || tag == ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
+ } else {
addListField(tmpValue, fieldType, fieldRecordBuilder, level + 1);
}
}
@@ -159,8 +157,7 @@
recordAccessor.getOpenFieldValue(recType, i, tmpValue.getDataOutput());
if (tag == ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
addNestedField(tmpValue, fieldType, fieldRecordBuilder, level + 1);
- } else if (tag == ATypeTag.SERIALIZED_ORDEREDLIST_TYPE_TAG
- || tag == ATypeTag.SERIALIZED_UNORDEREDLIST_TYPE_TAG) {
+ } else {
addListField(tmpValue, fieldType, fieldRecordBuilder, level + 1);
}
}
@@ -256,7 +253,7 @@
ArrayBackedValueStorage itemValue = getTempBuffer();
IARecordBuilder listRecordBuilder = getRecordBuilder();
- AListPointable list = getListPointable();
+ AListPointable list = getListPointable(fieldType.getTypeTag());
list.set(listArg);
OrderedListBuilder innerListBuilder = getOrderedListBuilder();
@@ -274,6 +271,7 @@
if (tagId == ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
ArrayBackedValueStorage tmpAbvs = getTempBuffer();
+ tmpAbvs.reset();
list.getItemValue(act, l, tmpAbvs.getDataOutput());
addNestedField(tmpAbvs, act.getItemType(), listRecordBuilder, level + 1);
}
@@ -288,8 +286,8 @@
return (ARecordPointable) recordPointablePool.allocate(ATypeTag.OBJECT);
}
- private AListPointable getListPointable() {
- return (AListPointable) listPointablePool.allocate(ATypeTag.ARRAY);
+ private AListPointable getListPointable(ATypeTag tag) {
+ return (AListPointable) listPointablePool.allocate(tag);
}
private IARecordBuilder getRecordBuilder() {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/utils/RandomHelper.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/utils/RandomHelper.java
index 7ed1ce5..bc905ba 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/utils/RandomHelper.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/utils/RandomHelper.java
@@ -20,7 +20,7 @@
package org.apache.asterix.runtime.evaluators.functions.utils;
import java.io.DataOutput;
-import java.io.IOException;
+import java.security.NoSuchAlgorithmException;
import java.security.SecureRandom;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
@@ -30,44 +30,42 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.api.IPointable;
import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.data.std.util.DataUtils;
-import org.apache.hyracks.data.std.util.GrowableArray;
public final class RandomHelper {
- private final SecureRandom random = new SecureRandom();
+ private final SecureRandom random;
- private final GrowableArray seed;
+ private double seed;
+
+ private boolean isFirst;
private final AMutableDouble aDouble = new AMutableDouble(0);
@SuppressWarnings("rawtypes")
- private ISerializerDeserializer doubleSerde =
+ private final ISerializerDeserializer doubleSerde =
SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
private final DataOutput dataOutput = resultStorage.getDataOutput();
- public RandomHelper(boolean withSeed) {
- seed = withSeed ? new GrowableArray(8) : null;
+ public RandomHelper(boolean withSeed) throws HyracksDataException {
+ if (withSeed) {
+ try {
+ random = SecureRandom.getInstance("SHA1PRNG");
+ } catch (NoSuchAlgorithmException e) {
+ throw new IllegalStateException("random()");
+ }
+ } else {
+ random = new SecureRandom();
+ }
+ isFirst = true;
}
- public void setSeed(byte[] bytes, int offset, int length) throws HyracksDataException {
- if (seed == null) {
- throw new IllegalStateException();
- }
-
- boolean sameSeed =
- seed.getLength() == length && DataUtils.equalsInRange(seed.getByteArray(), 0, bytes, offset, length);
-
- if (!sameSeed) {
- try {
- seed.reset();
- seed.append(bytes, offset, length);
- random.setSeed(seed.getByteArray());
- } catch (IOException e) {
- throw HyracksDataException.create(e);
- }
+ public void setSeed(double seedVal) throws HyracksDataException {
+ if (isFirst || seedVal != seed) {
+ seed = seedVal;
+ isFirst = false;
+ random.setSeed(Double.doubleToLongBits(seedVal));
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/InvalidDataFormatException.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/InvalidDataFormatException.java
index bb94ca6..ba713e3 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/InvalidDataFormatException.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/InvalidDataFormatException.java
@@ -28,19 +28,17 @@
public class InvalidDataFormatException extends RuntimeDataException {
private static final long serialVersionUID = 7927137063741221011L;
- public InvalidDataFormatException(SourceLocation sourceLoc, FunctionIdentifier fid, byte expectedTypeTag) {
- super(ErrorCode.INVALID_FORMAT, sourceLoc, fid.getName(),
+ public InvalidDataFormatException(SourceLocation sourceLoc, FunctionIdentifier fid, Throwable cause,
+ byte expectedTypeTag) {
+ super(ErrorCode.INVALID_FORMAT, cause, sourceLoc, fid.getName(),
EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(expectedTypeTag));
}
+ public InvalidDataFormatException(SourceLocation sourceLoc, FunctionIdentifier fid, byte expectedTypeTag) {
+ this(sourceLoc, fid, null, expectedTypeTag);
+ }
+
public InvalidDataFormatException(SourceLocation sourceLoc, FunctionIdentifier fid, String expectedType) {
super(ErrorCode.INVALID_FORMAT, sourceLoc, fid.getName(), expectedType);
}
-
- public InvalidDataFormatException(SourceLocation sourceLoc, FunctionIdentifier fid, Throwable cause,
- byte expectedTypeTag) {
- super(ErrorCode.INVALID_FORMAT, sourceLoc, fid.getName(), cause, expectedTypeTag);
- addSuppressed(cause);
- }
-
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/TypeMismatchException.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/TypeMismatchException.java
index e510fba..f51c305 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/TypeMismatchException.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/exceptions/TypeMismatchException.java
@@ -32,14 +32,14 @@
private static final long serialVersionUID = -668005043013338591L;
// Function parameter type mismatch.
- public TypeMismatchException(FunctionIdentifier fid, Integer i, byte actualTypeTag, byte... expectedTypeTags) {
+ public TypeMismatchException(FunctionIdentifier fid, int i, byte actualTypeTag, byte... expectedTypeTags) {
super(ErrorCode.TYPE_MISMATCH_FUNCTION, fid.getName(), indexToPosition(i),
toExpectedTypeString(expectedTypeTags),
EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(actualTypeTag));
}
// Function parameter type mismatch.
- public TypeMismatchException(SourceLocation sourceLoc, FunctionIdentifier fid, Integer i, byte actualTypeTag,
+ public TypeMismatchException(SourceLocation sourceLoc, FunctionIdentifier fid, int i, byte actualTypeTag,
byte... expectedTypeTags) {
super(ErrorCode.TYPE_MISMATCH_FUNCTION, sourceLoc, fid.getName(), indexToPosition(i),
toExpectedTypeString(expectedTypeTags),
@@ -47,7 +47,7 @@
}
// Function parameter type mismatch.
- public TypeMismatchException(SourceLocation sourceLoc, FunctionIdentifier fid, Integer i, byte actualTypeTag,
+ public TypeMismatchException(SourceLocation sourceLoc, FunctionIdentifier fid, int i, byte actualTypeTag,
String expectedType) {
super(ErrorCode.TYPE_MISMATCH_FUNCTION, sourceLoc, fid.getName(), indexToPosition(i), expectedType,
EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(actualTypeTag));
@@ -55,7 +55,7 @@
// Function parameter type mismatch.
@Deprecated
- public TypeMismatchException(String functionName, Integer i, byte actualTypeTag, byte... expectedTypeTags) {
+ public TypeMismatchException(String functionName, int i, byte actualTypeTag, byte... expectedTypeTags) {
super(ErrorCode.TYPE_MISMATCH_FUNCTION, functionName, indexToPosition(i),
toExpectedTypeString(expectedTypeTags),
EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(actualTypeTag));
@@ -63,7 +63,7 @@
// Function parameter type mismatch.
@Deprecated
- public TypeMismatchException(SourceLocation sourceLoc, String functionName, Integer i, byte actualTypeTag,
+ public TypeMismatchException(SourceLocation sourceLoc, String functionName, int i, byte actualTypeTag,
byte... expectedTypeTags) {
super(ErrorCode.TYPE_MISMATCH_FUNCTION, sourceLoc, functionName, indexToPosition(i),
toExpectedTypeString(expectedTypeTags),
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java
index aa5775e..35ae904 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertOperatorNodePushable.java
@@ -45,21 +45,20 @@
* This operator node is used for secondary indexes with upsert operations.
* It works in the following way:
* For each incoming tuple
- * -If old secondary keys == new secondary keys
+ * -If old secondary index tuple == new secondary index tuple
* --do nothing
* -else
- * --If old secondary keys are null?
+ * --If any old field is null/missing?
* ---do nothing
* --else
- * ---delete old secondary keys
- * --If new keys are null?
+ * ---delete old secondary index tuple
+ * --If any new field is null/missing?
* ---do nothing
* --else
- * ---insert new keys
+ * ---insert new secondary index tuple
*/
public class LSMSecondaryUpsertOperatorNodePushable extends LSMIndexInsertUpdateDeleteOperatorNodePushable {
- private static final int NULL_MISSING_FIELD_INDEX = 0;
private final PermutingFrameTupleReference prevValueTuple = new PermutingFrameTupleReference();
private final int upsertIndicatorFieldIndex;
private final IBinaryBooleanInspector upsertIndicatorInspector;
@@ -105,9 +104,9 @@
tuple.reset(accessor, i);
prevValueTuple.reset(accessor, i);
- boolean isNewValueNullOrMissing = isNullOrMissing(tuple);
- boolean isOldValueNullOrMissing = isNullOrMissing(prevValueTuple);
- if (isNewValueNullOrMissing && isOldValueNullOrMissing) {
+ boolean newTupleHasNullOrMissing = hasNullOrMissing(tuple);
+ boolean oldTupleHasNullOrMissing = hasNullOrMissing(prevValueTuple);
+ if (newTupleHasNullOrMissing && oldTupleHasNullOrMissing) {
// No op
continue;
}
@@ -118,13 +117,13 @@
// which are always the same
continue;
}
- if (!isOldValueNullOrMissing) {
- // We need to delete previous
+ // if all old fields are known values, then delete. skip deleting if any is null or missing
+ if (!oldTupleHasNullOrMissing) {
abstractModCallback.setOp(Operation.DELETE);
lsmAccessor.forceDelete(prevValueTuple);
}
- if (isUpsert && !isNewValueNullOrMissing) {
- // we need to insert the new value
+ // if all new fields are known values, then insert. skip inserting if any is null or missing
+ if (isUpsert && !newTupleHasNullOrMissing) {
abstractModCallback.setOp(Operation.INSERT);
lsmAccessor.forceInsert(tuple);
}
@@ -138,8 +137,18 @@
FrameUtils.flushFrame(writeBuffer.getBuffer(), writer);
}
- private static boolean isNullOrMissing(PermutingFrameTupleReference tuple) {
- return TypeTagUtil.isType(tuple, NULL_MISSING_FIELD_INDEX, ATypeTag.SERIALIZED_NULL_TYPE_TAG)
- || TypeTagUtil.isType(tuple, NULL_MISSING_FIELD_INDEX, ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
+ private boolean hasNullOrMissing(PermutingFrameTupleReference tuple) {
+ int fieldCount = tuple.getFieldCount();
+ for (int i = 0; i < fieldCount; i++) {
+ if (isNullOrMissing(tuple, i)) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ private static boolean isNullOrMissing(PermutingFrameTupleReference tuple, int fieldIdx) {
+ return TypeTagUtil.isType(tuple, fieldIdx, ATypeTag.SERIALIZED_NULL_TYPE_TAG)
+ || TypeTagUtil.isType(tuple, fieldIdx, ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
}
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
index a0b10c6..3366ac1 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/CcApplicationContext.java
@@ -45,6 +45,7 @@
import org.apache.asterix.common.config.TransactionProperties;
import org.apache.asterix.common.context.IStorageComponentProvider;
import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.external.IAdapterFactoryService;
import org.apache.asterix.common.library.ILibraryManager;
import org.apache.asterix.common.metadata.IMetadataBootstrap;
import org.apache.asterix.common.replication.INcLifecycleCoordinator;
@@ -98,14 +99,15 @@
private final IReceptionist receptionist;
private final IRequestTracker requestTracker;
private final IConfigValidator configValidator;
+ private final IAdapterFactoryService adapterFactoryService;
public CcApplicationContext(ICCServiceContext ccServiceCtx, IHyracksClientConnection hcc,
ILibraryManager libraryManager, Supplier<IMetadataBootstrap> metadataBootstrapSupplier,
IGlobalRecoveryManager globalRecoveryManager, INcLifecycleCoordinator ftStrategy,
IJobLifecycleListener activeLifeCycleListener, IStorageComponentProvider storageComponentProvider,
IMetadataLockManager mdLockManager, IReceptionistFactory receptionistFactory,
- IConfigValidatorFactory configValidatorFactory, Object extensionManager)
- throws AlgebricksException, IOException {
+ IConfigValidatorFactory configValidatorFactory, Object extensionManager,
+ IAdapterFactoryService adapterFactoryService) throws AlgebricksException, IOException {
this.ccServiceCtx = ccServiceCtx;
this.hcc = hcc;
this.libraryManager = libraryManager;
@@ -139,6 +141,7 @@
receptionist = receptionistFactory.create();
requestTracker = new RequestTracker(this);
configValidator = configValidatorFactory.create();
+ this.adapterFactoryService = adapterFactoryService;
}
@Override
@@ -306,4 +309,9 @@
public IRequestTracker getRequestTracker() {
return requestTracker;
}
+
+ @Override
+ public IAdapterFactoryService getAdapterFactoryService() {
+ return adapterFactoryService;
+ }
}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/NoOpCoordinationService.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/NoOpCoordinationService.java
index 6f9a8d2..3cbaed4 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/NoOpCoordinationService.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/utils/NoOpCoordinationService.java
@@ -35,6 +35,11 @@
}
@Override
+ public void putSensitive(String key, byte[] value) {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
public Optional<byte[]> get(String key) {
throw new UnsupportedOperationException();
}
diff --git a/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/job/resource/JobCapacityControllerTest.java b/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/job/resource/JobCapacityControllerTest.java
index 4a63885..48c61b4 100644
--- a/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/job/resource/JobCapacityControllerTest.java
+++ b/asterixdb/asterix-runtime/src/test/java/org/apache/asterix/runtime/job/resource/JobCapacityControllerTest.java
@@ -53,7 +53,7 @@
try {
capacityController.allocate(makeJobWithRequiredCapacity(2147483648L, 64));
} catch (HyracksException e) {
- exceedCapacity = e.getErrorCode() == ErrorCode.JOB_REQUIREMENTS_EXCEED_CAPACITY;
+ exceedCapacity = e.matches(ErrorCode.JOB_REQUIREMENTS_EXCEED_CAPACITY);
}
Assert.assertTrue(exceedCapacity);
Assert.assertTrue(capacityController.allocate(
@@ -62,7 +62,7 @@
try {
capacityController.allocate(makeJobWithRequiredCapacity(4294967297L, 33));
} catch (HyracksException e) {
- exceedCapacity = e.getErrorCode() == ErrorCode.JOB_REQUIREMENTS_EXCEED_CAPACITY;
+ exceedCapacity = e.matches(ErrorCode.JOB_REQUIREMENTS_EXCEED_CAPACITY);
}
Assert.assertTrue(exceedCapacity);
diff --git a/asterixdb/asterix-server/pom.xml b/asterixdb/asterix-server/pom.xml
index e912257..a3fed30 100644
--- a/asterixdb/asterix-server/pom.xml
+++ b/asterixdb/asterix-server/pom.xml
@@ -185,6 +185,35 @@
<url>http://www.apache.org/licenses/LICENSE-2.0</url>
<noticeUrl>https://raw.githubusercontent.com/netty/netty-tcnative/netty-tcnative-parent-2.0.46.Final/NOTICE.txt</noticeUrl>
</override>
+ <override>
+ <gav>org.reactivestreams:reactive-streams:1.0.2</gav>
+ <noticeUrl>https://raw.githubusercontent.com/reactive-streams/reactive-streams-jvm/v1.0.2/COPYING.txt</noticeUrl>
+ <url>https://raw.githubusercontent.com/reactive-streams/reactive-streams-jvm/v1.0.2/LICENSE.txt</url>
+ </override>
+ <override>
+ <gavs>
+ <gav>software.amazon.awssdk:sdk-core:2.10.83</gav>
+ <gav>software.amazon.awssdk:aws-core:2.10.83</gav>
+ <gav>software.amazon.awssdk:auth:2.10.83</gav>
+ <gav>software.amazon.awssdk:arns:2.10.83</gav>
+ <gav>software.amazon.awssdk:annotations:2.10.83</gav>
+ <gav>software.amazon.awssdk:utils:2.10.83</gav>
+ <gav>software.amazon.awssdk:aws-query-protocol:2.10.83</gav>
+ <gav>software.amazon.awssdk:profiles:2.10.83</gav>
+ <gav>software.amazon.awssdk:protocol-core:2.10.83</gav>
+ <gav>software.amazon.awssdk:s3:2.10.83</gav>
+ <gav>software.amazon.awssdk:netty-nio-client:2.10.83</gav>
+ <gav>software.amazon.awssdk:apache-client:2.10.83</gav>
+ <gav>software.amazon.awssdk:aws-xml-protocol:2.10.83</gav>
+ <gav>software.amazon.awssdk:regions:2.10.83</gav>
+ <gav>software.amazon.awssdk:http-client-spi:2.10.83</gav>
+ </gavs>
+ <noticeUrl>https://raw.githubusercontent.com/aws/aws-sdk-java-v2/2.10.83/NOTICE.txt</noticeUrl>
+ </override>
+ <override>
+ <gav>software.amazon.eventstream:eventstream:1.0.1</gav>
+ <noticeUrl>https://raw.githubusercontent.com/awslabs/aws-eventstream-java/7be2dd80e12f8835674c8ffb0f4a2efb64c7b585/NOTICE</noticeUrl>
+ </override>
</overrides>
<licenses>
<license>
@@ -219,6 +248,7 @@
<aliasUrl>https://www.apache.org/licenses/LICENSE-2.0.txt</aliasUrl>
<aliasUrl>http://www.apache.org/licenses/LICENSE-2.0.html</aliasUrl>
<aliasUrl>https://www.apache.org/licenses/LICENSE-2.0.html</aliasUrl>
+ <aliasUrl>https://aws.amazon.com/apache2.0</aliasUrl>
</aliasUrls>
<metric>1</metric>
</license>
@@ -278,6 +308,13 @@
<url>PUBLIC_DOMAIN</url>
<aliasUrls>Public Domain</aliasUrls>
</license>
+ <license>
+ <url>https://creativecommons.org/publicdomain/zero/1.0/legalcode.txt</url>
+ <aliasUrls>https://creativecommons.org/publicdomain/zero/1.0/legalcode</aliasUrls>
+ <aliasUrls>https://creativecommons.org/publicdomain/zero/1.0/</aliasUrls>
+ <aliasUrls>http://creativecommons.org/publicdomain/zero/1.0/legalcode</aliasUrls>
+ <aliasUrls>http://creativecommons.org/publicdomain/zero/1.0/</aliasUrls>
+ </license>
</licenses>
<templateProperties>
<packageName>Apache AsterixDB Server Install</packageName>
@@ -699,5 +736,13 @@
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
</dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>algebricks-core</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-api</artifactId>
+ </dependency>
</dependencies>
</project>
diff --git a/asterixdb/asterix-server/src/test/java/org/apache/asterix/test/server/FormattedExceptionInstanceTest.java b/asterixdb/asterix-server/src/test/java/org/apache/asterix/test/server/FormattedExceptionInstanceTest.java
new file mode 100644
index 0000000..f8d7d03
--- /dev/null
+++ b/asterixdb/asterix-server/src/test/java/org/apache/asterix/test/server/FormattedExceptionInstanceTest.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.test.server;
+
+import java.lang.reflect.Executable;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.api.exceptions.IFormattedException;
+import org.apache.hyracks.test.support.FormattedExceptionTestBase;
+import org.junit.runners.Parameterized;
+
+public class FormattedExceptionInstanceTest extends FormattedExceptionTestBase {
+ private static final ErrorCode ASK_ERROR_CODE = random(ErrorCode.values());
+ private static final FunctionIdentifier FUNCTION_IDENTIFIER = new FunctionIdentifier("fake", "fake");
+ static {
+ classSelector = classname -> classname.matches("^org\\.apache\\.(asterix|hyracks)\\..*");
+ }
+
+ public FormattedExceptionInstanceTest(String desc, Executable action, Class<? extends IFormattedException> root) {
+ super(desc, action, root);
+ }
+
+ @Parameterized.Parameters(name = "{index}: {0}")
+ public static Iterable<Object[]> data() throws ClassNotFoundException {
+ return defineParameters();
+ }
+
+ @Override
+ protected Object defaultValue(Class type) {
+ switch (type.getName()) {
+ case "org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier":
+ return FUNCTION_IDENTIFIER;
+ case "org.apache.asterix.common.exceptions.ErrorCode":
+ return ASK_ERROR_CODE;
+ default:
+ return super.defaultValue(type);
+ }
+ }
+}
diff --git a/asterixdb/asterix-spidersilk/config/cc.conf b/asterixdb/asterix-spidersilk/config/cc.conf
deleted file mode 100644
index 3212003..0000000
--- a/asterixdb/asterix-spidersilk/config/cc.conf
+++ /dev/null
@@ -1,35 +0,0 @@
-; Licensed to the Apache Software Foundation (ASF) under one
-; or more contributor license agreements. See the NOTICE file
-; distributed with this work for additional information
-; regarding copyright ownership. The ASF licenses this file
-; to you under the Apache License, Version 2.0 (the
-; "License"); you may not use this file except in compliance
-; with the License. You may obtain a copy of the License at
-;
-; http://www.apache.org/licenses/LICENSE-2.0
-;
-; Unless required by applicable law or agreed to in writing,
-; software distributed under the License is distributed on an
-; "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-; KIND, either express or implied. See the License for the
-; specific language governing permissions and limitations
-; under the License.
-
-[nc/nc1]
-txn.log.dir=/data/txnlog
-core.dump.dir=/data/coredump
-iodevices=/data
-address=nc1
-
-[nc/nc2]
-txn.log.dir=/data/txnlog
-core.dump.dir=/data/coredump
-iodevices=/data
-address=nc2
-
-[nc]
-app.class=org.apache.asterix.hyracks.bootstrap.NCApplicationEntryPoint
-command=asterixnc
-
-[cc]
-address=cc
\ No newline at end of file
diff --git a/asterixdb/asterix-spidersilk/docker/Dockerfile b/asterixdb/asterix-spidersilk/docker/Dockerfile
deleted file mode 100644
index b25561a..0000000
--- a/asterixdb/asterix-spidersilk/docker/Dockerfile
+++ /dev/null
@@ -1,22 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing,
-# software distributed under the License is distributed on an
-# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-# KIND, either express or implied. See the License for the
-# specific language governing permissions and limitations
-# under the License.
-
-FROM centos:7
-
-USER root
-
-RUN yum install -y unzip git which docker-client java-1.8.0-openjdk-headless
diff --git a/asterixdb/asterix-spidersilk/pom.xml b/asterixdb/asterix-spidersilk/pom.xml
index d3c10c6..c58ee4f 100644
--- a/asterixdb/asterix-spidersilk/pom.xml
+++ b/asterixdb/asterix-spidersilk/pom.xml
@@ -35,89 +35,4 @@
<comments>A business-friendly OSS license</comments>
</license>
</licenses>
-
- <properties>
- <root.dir>${basedir}/..</root.dir>
- </properties>
-
- <repositories>
- <repository>
- <id>snapshots-repo</id>
- <url>https://oss.sonatype.org/content/repositories/snapshots</url>
- <releases><enabled>false</enabled></releases>
- <snapshots><enabled>true</enabled></snapshots>
- </repository>
- </repositories>
-
- <dependencies>
- <dependency>
- <groupId>junit</groupId>
- <artifactId>junit</artifactId>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>me.arminb.spidersilk</groupId>
- <artifactId>spidersilk</artifactId>
- <version>0.4.0</version>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-api</artifactId>
- <version>1.7.33</version>
- </dependency>
- <dependency>
- <groupId>com.fasterxml.jackson.core</groupId>
- <artifactId>jackson-databind</artifactId>
- </dependency>
- <dependency>
- <groupId>ch.qos.logback</groupId>
- <artifactId>logback-classic</artifactId>
- <version>1.2.3</version>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.asterix</groupId>
- <artifactId>asterix-app</artifactId>
- <version>${project.version}</version>
- <type>test-jar</type>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.asterix</groupId>
- <artifactId>asterix-test-framework</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.asterix</groupId>
- <artifactId>asterix-app</artifactId>
- <version>${project.version}</version>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.asterix</groupId>
- <artifactId>asterix-server</artifactId>
- <version>${project.version}</version>
- </dependency>
-
- </dependencies>
-
- <!-- While these dependencies are declared and being used, the maven dependency plugin detects them as declared and
- not used. To make this right, it is needed to force these dependencies as used -->
- <build>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-dependency-plugin</artifactId>
- <configuration>
- <usedDependencies>
- <usedDependency>ch.qos.logback:logback-classic</usedDependency>
- <usedDependency>org.apache.asterix:asterix-app</usedDependency>
- <usedDependency>org.apache.asterix:asterix-server</usedDependency>
- </usedDependencies>
- </configuration>
- </plugin>
- </plugins>
- </build>
</project>
diff --git a/asterixdb/asterix-spidersilk/src/test/java/org/apache/asterix/spidersilk/SampleTestIT.java b/asterixdb/asterix-spidersilk/src/test/java/org/apache/asterix/spidersilk/SampleTestIT.java
deleted file mode 100644
index b3b73bf..0000000
--- a/asterixdb/asterix-spidersilk/src/test/java/org/apache/asterix/spidersilk/SampleTestIT.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.spidersilk;
-
-import java.io.InputStream;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.asterix.test.common.TestExecutor;
-import org.apache.asterix.testframework.context.TestCaseContext;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import me.arminb.spidersilk.SpiderSilkRunner;
-import me.arminb.spidersilk.dsl.entities.Deployment;
-import me.arminb.spidersilk.exceptions.RuntimeEngineException;
-
-public class SampleTestIT {
- private static final Logger logger = LoggerFactory.getLogger(SampleTestIT.class);
-
- protected static SpiderSilkRunner runner;
-
- @BeforeClass
- public static void before() throws RuntimeEngineException {
- Deployment deployment = TestUtil.getSimpleClusterDeployment();
- runner = SpiderSilkRunner.run(deployment);
- TestUtil.waitForClusterToBeUp(runner);
- logger.info("The cluster is UP!");
- }
-
- @AfterClass
- public static void after() {
- if (runner != null) {
- runner.stop();
- }
- }
-
- @Test
- public void sampleTest() throws Exception {
-
- TestExecutor testExecutor = TestUtil.getTestExecutor(runner);
- String ddl = "drop dataverse company if exists;" + "create dataverse company;" + "use company;"
- + "create type Emp as open {" + " id : int32," + " name : string" + "};"
- + "create dataset Employee(Emp) primary key id;";
-
- String insertStatements = "use company;" + "insert into Employee({ \"id\":123,\"name\":\"John Doe\"});";
-
- String query = "use company;" + "select value emp from Employee emp;";
-
- testExecutor.executeSqlppUpdateOrDdl(ddl, TestCaseContext.OutputFormat.CLEAN_JSON);
- logger.info("Company dataverse and employee dataset are created!");
- testExecutor.executeSqlppUpdateOrDdl(insertStatements, TestCaseContext.OutputFormat.CLEAN_JSON);
- logger.info("A record is inserted into employee dataset");
- InputStream resultStream = testExecutor.executeSqlppUpdateOrDdl(query, TestCaseContext.OutputFormat.CLEAN_JSON);
-
- ObjectMapper objectMapper = new ObjectMapper();
- List<Map<String, String>> result = objectMapper.readValue(resultStream, List.class);
-
- Assert.assertEquals(1, result.size());
- Assert.assertEquals(123, result.get(0).get("id"));
- Assert.assertEquals("John Doe", result.get(0).get("name"));
-
- logger.info("The fetched record matches the inserted record");
- }
-}
diff --git a/asterixdb/asterix-spidersilk/src/test/java/org/apache/asterix/spidersilk/TestUtil.java b/asterixdb/asterix-spidersilk/src/test/java/org/apache/asterix/spidersilk/TestUtil.java
deleted file mode 100644
index 03e8191..0000000
--- a/asterixdb/asterix-spidersilk/src/test/java/org/apache/asterix/spidersilk/TestUtil.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.spidersilk;
-
-import java.io.File;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.stream.Stream;
-
-import org.apache.asterix.test.common.TestExecutor;
-
-import me.arminb.spidersilk.SpiderSilkRunner;
-import me.arminb.spidersilk.dsl.entities.Deployment;
-import me.arminb.spidersilk.dsl.entities.PortType;
-import me.arminb.spidersilk.dsl.entities.ServiceType;
-import me.arminb.spidersilk.exceptions.RuntimeEngineException;
-
-public class TestUtil {
- private static String mavenVersion;
- private static String asterixHome;
-
- public static Deployment getSimpleClusterDeployment() {
- mavenVersion = getMavenArtifactVersion();
- asterixHome = "/asterix/apache-asterixdb-" + mavenVersion;
-
- return new Deployment.DeploymentBuilder("simpleClusterDeployment")
- // Service Definitions
- .withService("asterix")
- .applicationPath("../asterix-server/target/asterix-server-" + mavenVersion + "-binary-assembly.zip",
- "/asterix", false, true, false)
- .dockerFileAddress("docker/Dockerfile", false).dockerImage("spidersilk/test-asterix")
- .instrumentablePath(asterixHome + "/repo/asterix-server-" + mavenVersion + ".jar")
- .libraryPath(asterixHome + "/repo/*.jar").libraryPath(asterixHome + "/lib/*.jar")
- .logDirectory(asterixHome + "/logs").serviceType(ServiceType.JAVA).and()
- // Node Definitions
- .withNode("cc", "asterix").applicationPath("config", "/asterix/config")
- .startCommand(asterixHome + "/bin/asterixcc -config-file /asterix/config/cc.conf").tcpPort(19002).and()
- .withNode("nc1", "asterix").startCommand(asterixHome + "/bin/asterixncservice").and()
- .withNode("nc2", "asterix").startCommand(asterixHome + "/bin/asterixncservice").and().build();
- }
-
- public static String getMavenArtifactVersion() {
- Optional<String> version = Stream
- .of(Objects.requireNonNull(new File("../asterix-server/target")
- .list((dir, name) -> name.matches("asterix-server-.*-binary-assembly.zip"))))
- .map(foo -> foo.replace("asterix-server-", "")).map(foo -> foo.replace("-binary-assembly.zip", ""))
- .findFirst();
- return version.orElseThrow(IllegalStateException::new);
- }
-
- public static void waitForClusterToBeUp(SpiderSilkRunner runner) throws RuntimeEngineException {
- runner.runtime().runCommandInNode("cc", asterixHome + "/bin/asterixhelper wait_for_cluster");
- }
-
- public static TestExecutor getTestExecutor(SpiderSilkRunner runner) {
- return new TestExecutor(runner.runtime().ip("cc"), runner.runtime().portMapping("cc", 19002, PortType.TCP));
- }
-}
diff --git a/asterixdb/asterix-spidersilk/src/test/resources/logback.xml b/asterixdb/asterix-spidersilk/src/test/resources/logback.xml
deleted file mode 100644
index ca7aa79..0000000
--- a/asterixdb/asterix-spidersilk/src/test/resources/logback.xml
+++ /dev/null
@@ -1,35 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- ~ Licensed to the Apache Software Foundation (ASF) under one
- ~ or more contributor license agreements. See the NOTICE file
- ~ distributed with this work for additional information
- ~ regarding copyright ownership. The ASF licenses this file
- ~ to you under the Apache License, Version 2.0 (the
- ~ "License"); you may not use this file except in compliance
- ~ with the License. You may obtain a copy of the License at
- ~
- ~ http://www.apache.org/licenses/LICENSE-2.0
- ~
- ~ Unless required by applicable law or agreed to in writing,
- ~ software distributed under the License is distributed on an
- ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- ~ KIND, either express or implied. See the License for the
- ~ specific language governing permissions and limitations
- ~ under the License.
- -->
-
-<configuration>
-
- <appender name="Console" class="ch.qos.logback.core.ConsoleAppender">
- <layout class="ch.qos.logback.classic.PatternLayout">
- <Pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n</Pattern>
- </layout>
- </appender>
-
- <logger name="me.arminb" level="DEBUG"/>
- <logger name="org.apache.asterix" level="DEBUG"/>
-
- <root level="ERROR">
- <appender-ref ref="Console" />
- </root>
-</configuration>
diff --git a/asterixdb/asterix-transactions/pom.xml b/asterixdb/asterix-transactions/pom.xml
index 5b84162..72aaaee 100644
--- a/asterixdb/asterix-transactions/pom.xml
+++ b/asterixdb/asterix-transactions/pom.xml
@@ -110,7 +110,7 @@
</dependency>
<dependency>
<groupId>org.mockito</groupId>
- <artifactId>mockito-all</artifactId>
+ <artifactId>mockito-core</artifactId>
<scope>test</scope>
</dependency>
<dependency>
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
index 48b5b8e..b259809 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
@@ -317,10 +317,11 @@
.collect(Collectors.toSet());
}
- public DatasetResourceReference getLocalResourceReference(String absoluteFilePath) throws HyracksDataException {
+ public Optional<DatasetResourceReference> getLocalResourceReference(String absoluteFilePath)
+ throws HyracksDataException {
final String localResourcePath = StoragePathUtil.getIndexFileRelativePath(absoluteFilePath);
final LocalResource lr = get(localResourcePath);
- return DatasetResourceReference.of(lr);
+ return lr != null ? Optional.of(DatasetResourceReference.of(lr)) : Optional.empty();
}
/**
diff --git a/asterixdb/pom.xml b/asterixdb/pom.xml
index 8a94592..8c78dfc 100644
--- a/asterixdb/pom.xml
+++ b/asterixdb/pom.xml
@@ -46,7 +46,8 @@
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
<file.encoding>UTF-8</file.encoding>
<jvm.extraargs />
- <source.jdk.version>1.8</source.jdk.version>
+ <source.jdk.version>11</source.jdk.version>
+ <target.jdk.version>11</target.jdk.version>
<javac.xlint.value>all</javac.xlint.value>
<source-format.goal>format</source-format.goal>
<source-format.skip>false</source-format.skip>
@@ -82,6 +83,7 @@
<hadoop.version>2.8.5</hadoop.version>
<jacoco.version>0.7.6.201602180812</jacoco.version>
<log4j.version>2.17.1</log4j.version>
+ <awsjavasdk.version>2.17.116</awsjavasdk.version>
<implementation.title>Apache AsterixDB - ${project.name}</implementation.title>
<implementation.url>https://asterixdb.apache.org/</implementation.url>
@@ -195,7 +197,6 @@
<exclude>**/*.json</exclude>
<exclude>**/*.adm</exclude>
<exclude>**/*.template</exclude>
- <exclude>**/.SpiderSilkWorkingDirectory/**</exclude>
<exclude>asterix-installer/**</exclude> <!-- in case -DskipInstaller -->
</excludes>
</configuration>
@@ -263,10 +264,49 @@
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-enforcer-plugin</artifactId>
- <version>3.0.0-M1</version>
+ <artifactId>maven-antrun-plugin</artifactId>
<executions>
<execution>
+ <id>check-pom-packaging</id>
+ <phase>validate</phase>
+ <goals>
+ <goal>run</goal>
+ </goals>
+ <configuration>
+ <exportAntProperties>true</exportAntProperties>
+ <target xmlns:unless="ant:unless">
+ <condition property="skipPomEnforcement">
+ <not><equals arg1="${project.packaging}" arg2="pom"/></not>
+ </condition>
+ <echo message="will enforce non-existence of test source dir due to ${project.packaging} packaging" unless:set="skipPomEnforcement"/>
+ </target>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-enforcer-plugin</artifactId>
+ <executions>
+ <execution>
+ <id>ensure-no-tests-for-pom-packaging</id>
+ <phase>validate</phase>
+ <goals>
+ <goal>enforce</goal>
+ </goals>
+ <configuration>
+ <rules>
+ <requireFilesDontExist>
+ <files>
+ <file>${project.build.testSourceDirectory}</file>
+ </files>
+ <message>#### Tests cannot exist in projects with '${project.packaging}' packaging!</message>
+ </requireFilesDontExist>
+ </rules>
+ <skip>${skipPomEnforcement}</skip>
+ </configuration>
+ </execution>
+ <execution>
<id>enforce-versions</id>
<goals>
<goal>enforce</goal>
@@ -276,6 +316,20 @@
<requireMavenVersion>
<version>[3.3.9,)</version>
</requireMavenVersion>
+ <requireJavaVersion>
+ <version>[11,)</version>
+ </requireJavaVersion>
+ </rules>
+ </configuration>
+ </execution>
+ <execution>
+ <id>ensure-no-duplicate-deps</id>
+ <goals>
+ <goal>enforce</goal>
+ </goals>
+ <configuration>
+ <rules>
+ <banDuplicatePomDependencyVersions/>
</rules>
</configuration>
</execution>
@@ -490,7 +544,7 @@
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
- <version>2.21.0</version>
+ <version>3.0.0-M5</version>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
@@ -538,7 +592,7 @@
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-dependency-plugin</artifactId>
- <version>3.1.1</version>
+ <version>3.1.2</version>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
@@ -587,6 +641,11 @@
<artifactId>impsort-maven-plugin</artifactId>
<version>1.2.0</version>
</plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-enforcer-plugin</artifactId>
+ <version>3.0.0-M3</version>
+ </plugin>
</plugins>
</pluginManagement>
</build>
@@ -617,19 +676,6 @@
</properties>
</profile>
<profile>
- <id>skip-dashboard</id>
- <activation>
- <property>
- <name>skip.npm</name>
- <value>true</value>
- </property>
- </activation>
- <properties>
- <license.stage>none</license.stage>
- <resource.stage>none</resource.stage>
- </properties>
- </profile>
- <profile>
<id>invalid-tests</id>
<properties>
<invalid.tests />
@@ -644,7 +690,7 @@
</property>
</activation>
<properties>
- <test.heap.size>2048</test.heap.size>
+ <test.heap.size>4096</test.heap.size>
</properties>
</profile>
@@ -657,7 +703,7 @@
</property>
</activation>
<properties>
- <test.heap.size>3072</test.heap.size>
+ <test.heap.size>4096</test.heap.size>
</properties>
</profile>
<profile>
@@ -806,40 +852,6 @@
<failIfNoTests>false</failIfNoTests>
</properties>
</profile>
- <profile>
- <id>java8</id>
- <activation>
- <jdk>[1.8,)</jdk>
- </activation>
- <properties>
- <target.jdk.version>8</target.jdk.version>
- </properties>
- </profile>
- <profile>
- <id>java8-bootclasspath</id>
- <activation>
- <property>
- <name>java8-bootclasspath</name>
- </property>
- </activation>
- <properties>
- <target.jdk.version>8</target.jdk.version>
- </properties>
- <build>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-compiler-plugin</artifactId>
- <configuration>
- <compilerArgs combine.children="append">
- <arg>-bootclasspath</arg>
- <arg>${java8-bootclasspath}</arg>
- </compilerArgs>
- </configuration>
- </plugin>
- </plugins>
- </build>
- </profile>
</profiles>
<modules>
@@ -1341,6 +1353,94 @@
<artifactId>reflections</artifactId>
<version>0.9.12</version>
</dependency>
+ <dependency>
+ <groupId>software.amazon.awssdk</groupId>
+ <artifactId>s3</artifactId>
+ <version>${awsjavasdk.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-codec-http</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-codec-http2</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-codec</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-transport</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-common</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-buffer</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-handler</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-transport-native-epoll</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>software.amazon.awssdk</groupId>
+ <artifactId>regions</artifactId>
+ <version>${awsjavasdk.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>software.amazon.awssdk</groupId>
+ <artifactId>auth</artifactId>
+ <version>${awsjavasdk.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>software.amazon.awssdk</groupId>
+ <artifactId>aws-core</artifactId>
+ <version>${awsjavasdk.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>software.amazon.awssdk</groupId>
+ <artifactId>sdk-core</artifactId>
+ <version>${awsjavasdk.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>software.amazon.awssdk</groupId>
+ <artifactId>http-client-spi</artifactId>
+ <version>${awsjavasdk.version}</version>
+ </dependency>
+ <!-- Mock for AWS S3 -->
+ <dependency>
+ <groupId>io.findify</groupId>
+ <artifactId>s3mock_2.12</artifactId>
+ <version>0.2.5</version>
+ </dependency>
+ <!-- Needed for the s3 mock -->
+ <dependency>
+ <groupId>com.typesafe.akka</groupId>
+ <artifactId>akka-http-core_2.12</artifactId>
+ <version>10.1.0</version>
+ </dependency>
</dependencies>
</dependencyManagement>
diff --git a/asterixdb/src/main/appended-resources/supplemental-models.xml b/asterixdb/src/main/appended-resources/supplemental-models.xml
index 30749a4..61c1935 100644
--- a/asterixdb/src/main/appended-resources/supplemental-models.xml
+++ b/asterixdb/src/main/appended-resources/supplemental-models.xml
@@ -169,6 +169,30 @@
<supplement>
<project>
<groupId>io.netty</groupId>
+ <artifactId>netty-transport-classes-epoll</artifactId>
+ <properties>
+ <!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
+ <license.ignoreMissingEmbeddedLicense>4.1.72.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.72.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.712.Final</license.ignoreNoticeOverride>
+ </properties>
+ </project>
+ </supplement>
+ <supplement>
+ <project>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-transport-native-unix-common</artifactId>
+ <properties>
+ <!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
+ <license.ignoreMissingEmbeddedLicense>4.1.72.Final</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>4.1.72.Final</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>4.1.712.Final</license.ignoreNoticeOverride>
+ </properties>
+ </project>
+ </supplement>
+ <supplement>
+ <project>
+ <groupId>io.netty</groupId>
<artifactId>netty-codec</artifactId>
<properties>
<!-- netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
@@ -384,4 +408,72 @@
</properties>
</project>
</supplement>
+
+ <supplement>
+ <project>
+ <groupId>software.amazon.awssdk</groupId>
+ <artifactId>third-party-jackson-core</artifactId>
+ <properties>
+ <license.alternateNoticeFile>2.17.116:META-INF/NOTICE.txt</license.alternateNoticeFile>
+ <license.alternateLicenseFile>2.17.116:META-INF/LICENSE.txt</license.alternateLicenseFile>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- software.amazon.eventstream is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
+ <!-- license override not needed, ALv2 is specified in its pom.xml -->
+ <!-- see https://github.com/awslabs/aws-eventstream-java -->
+ <supplement>
+ <project>
+ <groupId>software.amazon.eventstream</groupId>
+ <artifactId>eventstream</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.0.1</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.0.1</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>1.0.1</license.ignoreNoticeOverride>
+ </properties>
+ </project>
+ </supplement>
+ <!-- AWS SDK end -->
+
+ <!-- com.typesafe.netty is ALv2, and does not contain any embedded LICENSE or NOTICE file -->
+ <!-- license override not needed, ALv2 is specified in its pom.xml -->
+ <!-- see https://github.com/playframework/netty-reactive-streams -->
+ <supplement>
+ <project>
+ <groupId>com.typesafe.netty</groupId>
+ <artifactId>netty-reactive-streams</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>2.0.5</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.0.5</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <supplement>
+ <project>
+ <groupId>com.typesafe.netty</groupId>
+ <artifactId>netty-reactive-streams-http</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>2.0.5</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>2.0.5</license.ignoreMissingEmbeddedNotice>
+ </properties>
+ </project>
+ </supplement>
+
+ <!-- org.reactivestreams does not contain any embedded LICENSE or NOTICE file -->
+ <!-- see https://github.com/reactive-streams/reactive-streams-jvm -->
+ <supplement>
+ <project>
+ <groupId>org.reactivestreams</groupId>
+ <artifactId>reactive-streams</artifactId>
+ <properties>
+ <license.ignoreMissingEmbeddedLicense>1.0.3</license.ignoreMissingEmbeddedLicense>
+ <license.ignoreMissingEmbeddedNotice>1.0.3</license.ignoreMissingEmbeddedNotice>
+ <license.ignoreNoticeOverride>1.0.3</license.ignoreNoticeOverride>
+ </properties>
+ </project>
+ </supplement>
+
+
</supplementalDataModels>
diff --git a/asterixdb/src/main/licenses/content/creativecommons.org_publicdomain_zero_1.0_legalcode.txt b/asterixdb/src/main/licenses/content/creativecommons.org_publicdomain_zero_1.0_legalcode.txt
new file mode 100644
index 0000000..1625c17
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/creativecommons.org_publicdomain_zero_1.0_legalcode.txt
@@ -0,0 +1,121 @@
+Creative Commons Legal Code
+
+CC0 1.0 Universal
+
+ CREATIVE COMMONS CORPORATION IS NOT A LAW FIRM AND DOES NOT PROVIDE
+ LEGAL SERVICES. DISTRIBUTION OF THIS DOCUMENT DOES NOT CREATE AN
+ ATTORNEY-CLIENT RELATIONSHIP. CREATIVE COMMONS PROVIDES THIS
+ INFORMATION ON AN "AS-IS" BASIS. CREATIVE COMMONS MAKES NO WARRANTIES
+ REGARDING THE USE OF THIS DOCUMENT OR THE INFORMATION OR WORKS
+ PROVIDED HEREUNDER, AND DISCLAIMS LIABILITY FOR DAMAGES RESULTING FROM
+ THE USE OF THIS DOCUMENT OR THE INFORMATION OR WORKS PROVIDED
+ HEREUNDER.
+
+Statement of Purpose
+
+The laws of most jurisdictions throughout the world automatically confer
+exclusive Copyright and Related Rights (defined below) upon the creator
+and subsequent owner(s) (each and all, an "owner") of an original work of
+authorship and/or a database (each, a "Work").
+
+Certain owners wish to permanently relinquish those rights to a Work for
+the purpose of contributing to a commons of creative, cultural and
+scientific works ("Commons") that the public can reliably and without fear
+of later claims of infringement build upon, modify, incorporate in other
+works, reuse and redistribute as freely as possible in any form whatsoever
+and for any purposes, including without limitation commercial purposes.
+These owners may contribute to the Commons to promote the ideal of a free
+culture and the further production of creative, cultural and scientific
+works, or to gain reputation or greater distribution for their Work in
+part through the use and efforts of others.
+
+For these and/or other purposes and motivations, and without any
+expectation of additional consideration or compensation, the person
+associating CC0 with a Work (the "Affirmer"), to the extent that he or she
+is an owner of Copyright and Related Rights in the Work, voluntarily
+elects to apply CC0 to the Work and publicly distribute the Work under its
+terms, with knowledge of his or her Copyright and Related Rights in the
+Work and the meaning and intended legal effect of CC0 on those rights.
+
+1. Copyright and Related Rights. A Work made available under CC0 may be
+protected by copyright and related or neighboring rights ("Copyright and
+Related Rights"). Copyright and Related Rights include, but are not
+limited to, the following:
+
+ i. the right to reproduce, adapt, distribute, perform, display,
+ communicate, and translate a Work;
+ ii. moral rights retained by the original author(s) and/or performer(s);
+iii. publicity and privacy rights pertaining to a person's image or
+ likeness depicted in a Work;
+ iv. rights protecting against unfair competition in regards to a Work,
+ subject to the limitations in paragraph 4(a), below;
+ v. rights protecting the extraction, dissemination, use and reuse of data
+ in a Work;
+ vi. database rights (such as those arising under Directive 96/9/EC of the
+ European Parliament and of the Council of 11 March 1996 on the legal
+ protection of databases, and under any national implementation
+ thereof, including any amended or successor version of such
+ directive); and
+vii. other similar, equivalent or corresponding rights throughout the
+ world based on applicable law or treaty, and any national
+ implementations thereof.
+
+2. Waiver. To the greatest extent permitted by, but not in contravention
+of, applicable law, Affirmer hereby overtly, fully, permanently,
+irrevocably and unconditionally waives, abandons, and surrenders all of
+Affirmer's Copyright and Related Rights and associated claims and causes
+of action, whether now known or unknown (including existing as well as
+future claims and causes of action), in the Work (i) in all territories
+worldwide, (ii) for the maximum duration provided by applicable law or
+treaty (including future time extensions), (iii) in any current or future
+medium and for any number of copies, and (iv) for any purpose whatsoever,
+including without limitation commercial, advertising or promotional
+purposes (the "Waiver"). Affirmer makes the Waiver for the benefit of each
+member of the public at large and to the detriment of Affirmer's heirs and
+successors, fully intending that such Waiver shall not be subject to
+revocation, rescission, cancellation, termination, or any other legal or
+equitable action to disrupt the quiet enjoyment of the Work by the public
+as contemplated by Affirmer's express Statement of Purpose.
+
+3. Public License Fallback. Should any part of the Waiver for any reason
+be judged legally invalid or ineffective under applicable law, then the
+Waiver shall be preserved to the maximum extent permitted taking into
+account Affirmer's express Statement of Purpose. In addition, to the
+extent the Waiver is so judged Affirmer hereby grants to each affected
+person a royalty-free, non transferable, non sublicensable, non exclusive,
+irrevocable and unconditional license to exercise Affirmer's Copyright and
+Related Rights in the Work (i) in all territories worldwide, (ii) for the
+maximum duration provided by applicable law or treaty (including future
+time extensions), (iii) in any current or future medium and for any number
+of copies, and (iv) for any purpose whatsoever, including without
+limitation commercial, advertising or promotional purposes (the
+"License"). The License shall be deemed effective as of the date CC0 was
+applied by Affirmer to the Work. Should any part of the License for any
+reason be judged legally invalid or ineffective under applicable law, such
+partial invalidity or ineffectiveness shall not invalidate the remainder
+of the License, and in such case Affirmer hereby affirms that he or she
+will not (i) exercise any of his or her remaining Copyright and Related
+Rights in the Work or (ii) assert any associated claims and causes of
+action with respect to the Work, in either case contrary to Affirmer's
+express Statement of Purpose.
+
+4. Limitations and Disclaimers.
+
+ a. No trademark or patent rights held by Affirmer are waived, abandoned,
+ surrendered, licensed or otherwise affected by this document.
+ b. Affirmer offers the Work as-is and makes no representations or
+ warranties of any kind concerning the Work, express, implied,
+ statutory or otherwise, including without limitation warranties of
+ title, merchantability, fitness for a particular purpose, non
+ infringement, or the absence of latent or other defects, accuracy, or
+ the present or absence of errors, whether or not discoverable, all to
+ the greatest extent permissible under applicable law.
+ c. Affirmer disclaims responsibility for clearing rights of other persons
+ that may apply to the Work or any use thereof, including without
+ limitation any person's Copyright and Related Rights in the Work.
+ Further, Affirmer disclaims responsibility for obtaining any necessary
+ consents, permissions or other rights required for any use of the
+ Work.
+ d. Affirmer understands and acknowledges that Creative Commons is not a
+ party to this document and has no duty or obligation with respect to
+ this CC0 or use of the Work.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_aws_aws-sdk-java-v2_2.10.83_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_aws_aws-sdk-java-v2_2.10.83_NOTICE.txt
new file mode 100644
index 0000000..f3c67c6
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_aws_aws-sdk-java-v2_2.10.83_NOTICE.txt
@@ -0,0 +1,16 @@
+AWS SDK for Java 2.0
+Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved.
+
+This product includes software developed by
+Amazon Technologies, Inc (http://www.amazon.com/).
+
+**********************
+THIRD PARTY COMPONENTS
+**********************
+This software includes third party software subject to the following copyrights:
+- XML parsing and utility functions from JetS3t - Copyright 2006-2009 James Murty.
+- PKCS#1 PEM encoded private key parsing and utility functions from oauth.googlecode.com - Copyright 1998-2010 AOL Inc.
+- Apache Commons Lang - https://github.com/apache/commons-lang
+- Netty Reactive Streams - https://github.com/playframework/netty-reactive-streams
+
+The licenses for these third party components are included in LICENSE.txt
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_awslabs_aws-eventstream-java_7be2dd80e12f8835674c8ffb0f4a2efb64c7b585_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_awslabs_aws-eventstream-java_7be2dd80e12f8835674c8ffb0f4a2efb64c7b585_NOTICE.txt
new file mode 100644
index 0000000..6822ad6
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_awslabs_aws-eventstream-java_7be2dd80e12f8835674c8ffb0f4a2efb64c7b585_NOTICE.txt
@@ -0,0 +1,2 @@
+AWS EventStream for Java
+Copyright 2017 Amazon.com, Inc. or its affiliates. All Rights Reserved.
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.46.Final_NOTICE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.46.Final_NOTICE.txt
deleted file mode 100644
index d2fda58..0000000
--- a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_netty_netty_netty-4.1.46.Final_NOTICE.txt
+++ /dev/null
@@ -1,248 +0,0 @@
-
- The Netty Project
- =================
-
-Please visit the Netty web site for more information:
-
- * https://netty.io/
-
-Copyright 2014 The Netty Project
-
-The Netty Project licenses this file to you under the Apache License,
-version 2.0 (the "License"); you may not use this file except in compliance
-with the License. You may obtain a copy of the License at:
-
- http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
-WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
-License for the specific language governing permissions and limitations
-under the License.
-
-Also, please refer to each LICENSE.<component>.txt file, which is located in
-the 'license' directory of the distribution file, for the license terms of the
-components that this product depends on.
-
--------------------------------------------------------------------------------
-This product contains the extensions to Java Collections Framework which has
-been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene:
-
- * LICENSE:
- * license/LICENSE.jsr166y.txt (Public Domain)
- * HOMEPAGE:
- * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/
- * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/
-
-This product contains a modified version of Robert Harder's Public Domain
-Base64 Encoder and Decoder, which can be obtained at:
-
- * LICENSE:
- * license/LICENSE.base64.txt (Public Domain)
- * HOMEPAGE:
- * http://iharder.sourceforge.net/current/java/base64/
-
-This product contains a modified portion of 'Webbit', an event based
-WebSocket and HTTP server, which can be obtained at:
-
- * LICENSE:
- * license/LICENSE.webbit.txt (BSD License)
- * HOMEPAGE:
- * https://github.com/joewalnes/webbit
-
-This product contains a modified portion of 'SLF4J', a simple logging
-facade for Java, which can be obtained at:
-
- * LICENSE:
- * license/LICENSE.slf4j.txt (MIT License)
- * HOMEPAGE:
- * http://www.slf4j.org/
-
-This product contains a modified portion of 'Apache Harmony', an open source
-Java SE, which can be obtained at:
-
- * NOTICE:
- * license/NOTICE.harmony.txt
- * LICENSE:
- * license/LICENSE.harmony.txt (Apache License 2.0)
- * HOMEPAGE:
- * http://archive.apache.org/dist/harmony/
-
-This product contains a modified portion of 'jbzip2', a Java bzip2 compression
-and decompression library written by Matthew J. Francis. It can be obtained at:
-
- * LICENSE:
- * license/LICENSE.jbzip2.txt (MIT License)
- * HOMEPAGE:
- * https://code.google.com/p/jbzip2/
-
-This product contains a modified portion of 'libdivsufsort', a C API library to construct
-the suffix array and the Burrows-Wheeler transformed string for any input string of
-a constant-size alphabet written by Yuta Mori. It can be obtained at:
-
- * LICENSE:
- * license/LICENSE.libdivsufsort.txt (MIT License)
- * HOMEPAGE:
- * https://github.com/y-256/libdivsufsort
-
-This product contains a modified portion of Nitsan Wakart's 'JCTools', Java Concurrency Tools for the JVM,
- which can be obtained at:
-
- * LICENSE:
- * license/LICENSE.jctools.txt (ASL2 License)
- * HOMEPAGE:
- * https://github.com/JCTools/JCTools
-
-This product optionally depends on 'JZlib', a re-implementation of zlib in
-pure Java, which can be obtained at:
-
- * LICENSE:
- * license/LICENSE.jzlib.txt (BSD style License)
- * HOMEPAGE:
- * http://www.jcraft.com/jzlib/
-
-This product optionally depends on 'Compress-LZF', a Java library for encoding and
-decoding data in LZF format, written by Tatu Saloranta. It can be obtained at:
-
- * LICENSE:
- * license/LICENSE.compress-lzf.txt (Apache License 2.0)
- * HOMEPAGE:
- * https://github.com/ning/compress
-
-This product optionally depends on 'lz4', a LZ4 Java compression
-and decompression library written by Adrien Grand. It can be obtained at:
-
- * LICENSE:
- * license/LICENSE.lz4.txt (Apache License 2.0)
- * HOMEPAGE:
- * https://github.com/jpountz/lz4-java
-
-This product optionally depends on 'lzma-java', a LZMA Java compression
-and decompression library, which can be obtained at:
-
- * LICENSE:
- * license/LICENSE.lzma-java.txt (Apache License 2.0)
- * HOMEPAGE:
- * https://github.com/jponge/lzma-java
-
-This product contains a modified portion of 'jfastlz', a Java port of FastLZ compression
-and decompression library written by William Kinney. It can be obtained at:
-
- * LICENSE:
- * license/LICENSE.jfastlz.txt (MIT License)
- * HOMEPAGE:
- * https://code.google.com/p/jfastlz/
-
-This product contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data
-interchange format, which can be obtained at:
-
- * LICENSE:
- * license/LICENSE.protobuf.txt (New BSD License)
- * HOMEPAGE:
- * https://github.com/google/protobuf
-
-This product optionally depends on 'Bouncy Castle Crypto APIs' to generate
-a temporary self-signed X.509 certificate when the JVM does not provide the
-equivalent functionality. It can be obtained at:
-
- * LICENSE:
- * license/LICENSE.bouncycastle.txt (MIT License)
- * HOMEPAGE:
- * http://www.bouncycastle.org/
-
-This product optionally depends on 'Snappy', a compression library produced
-by Google Inc, which can be obtained at:
-
- * LICENSE:
- * license/LICENSE.snappy.txt (New BSD License)
- * HOMEPAGE:
- * https://github.com/google/snappy
-
-This product optionally depends on 'JBoss Marshalling', an alternative Java
-serialization API, which can be obtained at:
-
- * LICENSE:
- * license/LICENSE.jboss-marshalling.txt (Apache License 2.0)
- * HOMEPAGE:
- * https://github.com/jboss-remoting/jboss-marshalling
-
-This product optionally depends on 'Caliper', Google's micro-
-benchmarking framework, which can be obtained at:
-
- * LICENSE:
- * license/LICENSE.caliper.txt (Apache License 2.0)
- * HOMEPAGE:
- * https://github.com/google/caliper
-
-This product optionally depends on 'Apache Commons Logging', a logging
-framework, which can be obtained at:
-
- * LICENSE:
- * license/LICENSE.commons-logging.txt (Apache License 2.0)
- * HOMEPAGE:
- * http://commons.apache.org/logging/
-
-This product optionally depends on 'Apache Log4J', a logging framework, which
-can be obtained at:
-
- * LICENSE:
- * license/LICENSE.log4j.txt (Apache License 2.0)
- * HOMEPAGE:
- * http://logging.apache.org/log4j/
-
-This product optionally depends on 'Aalto XML', an ultra-high performance
-non-blocking XML processor, which can be obtained at:
-
- * LICENSE:
- * license/LICENSE.aalto-xml.txt (Apache License 2.0)
- * HOMEPAGE:
- * http://wiki.fasterxml.com/AaltoHome
-
-This product contains a modified version of 'HPACK', a Java implementation of
-the HTTP/2 HPACK algorithm written by Twitter. It can be obtained at:
-
- * LICENSE:
- * license/LICENSE.hpack.txt (Apache License 2.0)
- * HOMEPAGE:
- * https://github.com/twitter/hpack
-
-This product contains a modified version of 'HPACK', a Java implementation of
-the HTTP/2 HPACK algorithm written by Cory Benfield. It can be obtained at:
-
- * LICENSE:
- * license/LICENSE.hyper-hpack.txt (MIT License)
- * HOMEPAGE:
- * https://github.com/python-hyper/hpack/
-
-This product contains a modified version of 'HPACK', a Java implementation of
-the HTTP/2 HPACK algorithm written by Tatsuhiro Tsujikawa. It can be obtained at:
-
- * LICENSE:
- * license/LICENSE.nghttp2-hpack.txt (MIT License)
- * HOMEPAGE:
- * https://github.com/nghttp2/nghttp2/
-
-This product contains a modified portion of 'Apache Commons Lang', a Java library
-provides utilities for the java.lang API, which can be obtained at:
-
- * LICENSE:
- * license/LICENSE.commons-lang.txt (Apache License 2.0)
- * HOMEPAGE:
- * https://commons.apache.org/proper/commons-lang/
-
-
-This product contains the Maven wrapper scripts from 'Maven Wrapper', that provides an easy way to ensure a user has everything necessary to run the Maven build.
-
- * LICENSE:
- * license/LICENSE.mvn-wrapper.txt (Apache License 2.0)
- * HOMEPAGE:
- * https://github.com/takari/maven-wrapper
-
-This product contains the dnsinfo.h header file, that provides a way to retrieve the system DNS configuration on MacOS.
-This private header is also used by Apple's open source
- mDNSResponder (https://opensource.apple.com/tarballs/mDNSResponder/).
-
- * LICENSE:
- * license/LICENSE.dnsinfo.txt (Apache License 2.0)
- * HOMEPAGE:
- * http://www.opensource.apple.com/source/configd/configd-453.19/dnsinfo/dnsinfo.h
\ No newline at end of file
diff --git a/asterixdb/src/main/licenses/content/raw.githubusercontent.com_reactive-streams_reactive-streams-jvm_v1.0.3_LICENSE.txt b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_reactive-streams_reactive-streams-jvm_v1.0.3_LICENSE.txt
new file mode 100644
index 0000000..eadae05
--- /dev/null
+++ b/asterixdb/src/main/licenses/content/raw.githubusercontent.com_reactive-streams_reactive-streams-jvm_v1.0.3_LICENSE.txt
@@ -0,0 +1,8 @@
+Licensed under Public Domain (CC0)
+
+To the extent possible under law, the person who associated CC0 with
+this code has waived all copyright and related or neighboring
+rights to this code.
+
+You should have received a copy of the CC0 legalcode along with this
+work. If not, see <http://creativecommons.org/publicdomain/zero/1.0/>.
\ No newline at end of file
diff --git a/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/AlgebricksException.java b/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/AlgebricksException.java
index 46e80be..f00161c 100644
--- a/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/AlgebricksException.java
+++ b/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/AlgebricksException.java
@@ -19,8 +19,10 @@
package org.apache.hyracks.algebricks.common.exceptions;
import java.io.Serializable;
+import java.util.Optional;
import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.IError;
import org.apache.hyracks.api.exceptions.IFormattedException;
import org.apache.hyracks.api.exceptions.SourceLocation;
import org.apache.hyracks.api.util.ErrorMessageUtil;
@@ -34,13 +36,23 @@
private final Serializable[] params;
private final String nodeId;
private final SourceLocation sourceLoc;
+ protected final transient IError error;
@SuppressWarnings("squid:S1165") // exception class not final
- private transient CachedMessage msgCache;
+ private transient volatile String msgCache;
- public AlgebricksException(String component, int errorCode, String message, Throwable cause,
+ public static AlgebricksException create(ErrorCode error, SourceLocation sourceLoc, Serializable... params) {
+ return new AlgebricksException(error, sourceLoc, params);
+ }
+
+ public static AlgebricksException create(ErrorCode error, Serializable... params) {
+ return create(error, null, params);
+ }
+
+ protected AlgebricksException(IError error, String component, int errorCode, String message, Throwable cause,
SourceLocation sourceLoc, String nodeId, Serializable... params) {
super(message, cause);
+ this.error = error;
this.component = component;
this.errorCode = errorCode;
this.sourceLoc = sourceLoc;
@@ -53,7 +65,7 @@
*/
@Deprecated
public AlgebricksException(String message) {
- this(ErrorMessageUtil.NONE, UNKNOWN, message, null, null, (Serializable[]) null);
+ this((IError) null, ErrorMessageUtil.NONE, UNKNOWN, message, null, null, null);
}
/**
@@ -61,7 +73,7 @@
*/
@Deprecated
public AlgebricksException(Throwable cause) {
- this(ErrorMessageUtil.NONE, UNKNOWN, cause.getMessage(), cause, (Serializable[]) null);
+ this(String.valueOf(cause), cause);
}
/**
@@ -69,60 +81,23 @@
*/
@Deprecated
public AlgebricksException(String message, Throwable cause) {
- this(ErrorMessageUtil.NONE, UNKNOWN, message, cause, null, (Serializable[]) null);
+ this((IError) null, ErrorMessageUtil.NONE, UNKNOWN, message, cause, null, null);
}
- public AlgebricksException(String component, int errorCode, SourceLocation sourceLoc, Serializable... params) {
- this(component, errorCode, null, null, sourceLoc, null, params);
+ public AlgebricksException(Throwable cause, ErrorCode error, Serializable... params) {
+ this(error, error.component(), error.intValue(), error.errorMessage(), cause, null, null, params);
}
- public AlgebricksException(String component, int errorCode, Serializable... params) {
- this(component, errorCode, null, null, null, null, params);
+ public AlgebricksException(ErrorCode error, SourceLocation sourceLoc, Serializable... params) {
+ this(error, error.component(), error.intValue(), error.errorMessage(), null, sourceLoc, null, params);
}
- public AlgebricksException(Throwable cause, int errorCode, SourceLocation sourceLoc, Serializable... params) {
- this(ErrorMessageUtil.NONE, errorCode, cause.getMessage(), cause, sourceLoc, null, params);
+ public AlgebricksException(ErrorCode error, Serializable... params) {
+ this(error, null, params);
}
- public AlgebricksException(Throwable cause, int errorCode, Serializable... params) {
- this(ErrorMessageUtil.NONE, errorCode, cause.getMessage(), cause, null, null, params);
- }
-
- public AlgebricksException(String component, int errorCode, String message, SourceLocation sourceLoc,
- Serializable... params) {
- this(component, errorCode, message, null, sourceLoc, null, params);
- }
-
- public AlgebricksException(String component, int errorCode, String message, Serializable... params) {
- this(component, errorCode, message, null, null, null, params);
- }
-
- public AlgebricksException(String component, int errorCode, Throwable cause, SourceLocation sourceLoc,
- Serializable... params) {
- this(component, errorCode, cause.getMessage(), cause, sourceLoc, null, params);
- }
-
- public AlgebricksException(String component, int errorCode, Throwable cause, Serializable... params) {
- this(component, errorCode, cause.getMessage(), cause, null, null, params);
- }
-
- public AlgebricksException(String component, int errorCode, String message, Throwable cause,
- SourceLocation sourceLoc, Serializable... params) {
- this(component, errorCode, message, cause, sourceLoc, null, params);
- }
-
- public AlgebricksException(String component, int errorCode, String message, Throwable cause,
- Serializable... params) {
- this(component, errorCode, message, cause, null, null, params);
- }
-
- public static AlgebricksException create(int errorCode, SourceLocation sourceLoc, Serializable... params) {
- return new AlgebricksException(ErrorCode.HYRACKS, errorCode, ErrorCode.getErrorMessage(errorCode), sourceLoc,
- params);
- }
-
- public static AlgebricksException create(int errorCode, Serializable... params) {
- return create(errorCode, null, params);
+ protected AlgebricksException(IError error, Throwable cause, SourceLocation sourceLoc, Serializable... params) {
+ this(error, error.component(), error.intValue(), error.errorMessage(), cause, sourceLoc, null, params);
}
@Override
@@ -135,7 +110,7 @@
return errorCode;
}
- public Object[] getParams() {
+ public Serializable[] getParams() {
return params;
}
@@ -148,19 +123,15 @@
}
@Override
- public String getMessage() {
- if (msgCache == null) {
- msgCache = new CachedMessage(
- ErrorMessageUtil.formatMessage(component, errorCode, super.getMessage(), sourceLoc, params));
- }
- return msgCache.message;
+ public Optional<IError> getError() {
+ return Optional.ofNullable(error);
}
- private static class CachedMessage {
- private final String message;
-
- private CachedMessage(String message) {
- this.message = message;
+ @Override
+ public String getMessage() {
+ if (msgCache == null) {
+ msgCache = ErrorMessageUtil.formatMessage(component, errorCode, super.getMessage(), sourceLoc, params);
}
+ return msgCache;
}
}
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml b/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml
index ff92d91..3617593 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/pom.xml
@@ -68,26 +68,22 @@
</dependency>
<dependency>
<groupId>org.mockito</groupId>
- <artifactId>mockito-all</artifactId>
- <version>1.10.19</version>
+ <artifactId>mockito-core</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.powermock</groupId>
- <artifactId>powermock-api-mockito</artifactId>
- <version>1.6.6</version>
+ <artifactId>powermock-api-mockito2</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.powermock</groupId>
<artifactId>powermock-core</artifactId>
- <version>1.6.6</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.powermock</groupId>
<artifactId>powermock-module-junit4</artifactId>
- <version>1.6.6</version>
<scope>test</scope>
</dependency>
<dependency>
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
index ab68701..b96d657 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/main/java/org/apache/hyracks/algebricks/compiler/api/HeuristicCompilerFactoryBuilder.java
@@ -105,7 +105,7 @@
normalizedKeyComputerFactoryProvider, expressionRuntimeProvider, expressionTypeComputer,
oc, expressionEvalSizeComputer, partialAggregationTypeComputer,
predEvaluatorFactoryProvider, physicalOptimizationConfig.getFrameSize(),
- clusterLocations, maxWarnings);
+ clusterLocations, warningCollector, maxWarnings, physicalOptimizationConfig);
PlanCompiler pc = new PlanCompiler(context);
return pc.compilePlan(plan, jobEventListenerFactory);
diff --git a/hyracks-fullstack/algebricks/algebricks-compiler/src/test/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleControllerTest.java b/hyracks-fullstack/algebricks/algebricks-compiler/src/test/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleControllerTest.java
index 18aa104..b568417 100644
--- a/hyracks-fullstack/algebricks/algebricks-compiler/src/test/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleControllerTest.java
+++ b/hyracks-fullstack/algebricks/algebricks-compiler/src/test/java/org/apache/hyracks/algebricks/compiler/rewriter/rulecontrollers/SequentialFirstRuleCheckFixpointRuleControllerTest.java
@@ -41,7 +41,7 @@
import org.powermock.modules.junit4.PowerMockRunner;
@RunWith(PowerMockRunner.class)
-@PowerMockIgnore("javax.management.*")
+@PowerMockIgnore({ "javax.management.*", "javax.xml.*", "com.sun.org.apache.xerces.*", "org.w3c.*", "org.xml.*" })
@PrepareForTest({ SequentialFirstRuleCheckFixpointRuleController.class, AbstractLogicalOperator.class })
public class SequentialFirstRuleCheckFixpointRuleControllerTest {
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DistinctOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DistinctOperator.java
index 9ac6659..673d80d 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DistinctOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DistinctOperator.java
@@ -52,8 +52,8 @@
@Override
public void recomputeSchema() {
- schema = new ArrayList<LogicalVariable>();
- schema.addAll(this.getDistinctByVarList());
+ schema = new ArrayList<>();
+ schema.addAll(getDistinctByVarList());
List<LogicalVariable> inputSchema = inputs.get(0).getValue().getSchema();
for (LogicalVariable var : inputSchema) {
if (!schema.contains(var)) {
@@ -66,13 +66,12 @@
public VariablePropagationPolicy getVariablePropagationPolicy() {
return new VariablePropagationPolicy() {
@Override
- public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources)
- throws AlgebricksException {
- /** make sure distinct key vars laid-out first */
+ public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources) {
+ /* make sure distinct key vars laid-out first */
for (LogicalVariable keyVar : getDistinctByVarList()) {
target.addVariable(keyVar);
}
- /** add other source vars */
+ /* add other source vars */
for (IOperatorSchema srcSchema : sources) {
for (LogicalVariable srcVar : srcSchema)
if (target.findVariable(srcVar) < 0) {
@@ -105,7 +104,7 @@
}
public List<LogicalVariable> getDistinctByVarList() {
- List<LogicalVariable> varList = new ArrayList<LogicalVariable>(expressions.size());
+ List<LogicalVariable> varList = new ArrayList<>(expressions.size());
for (Mutable<ILogicalExpression> eRef : expressions) {
ILogicalExpression e = eRef.getValue();
if (e.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
@@ -121,7 +120,7 @@
ILogicalExpression e = eRef.getValue();
if (e.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
VariableReferenceExpression v = (VariableReferenceExpression) e;
- if (v.getVariableReference() == var) {
+ if (v.getVariableReference().equals(var)) {
return true;
}
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnionAllOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnionAllOperator.java
index 9defb4f2..d394e2f 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnionAllOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnionAllOperator.java
@@ -19,6 +19,7 @@
package org.apache.hyracks.algebricks.core.algebra.operators.logical;
import java.util.ArrayList;
+import java.util.LinkedHashSet;
import java.util.List;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -77,25 +78,34 @@
@Override
public void recomputeSchema() {
- schema = new ArrayList<LogicalVariable>();
- for (LogicalVariable v1 : inputs.get(0).getValue().getSchema()) {
- for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> t : varMap) {
- if (t.first.equals(v1)) {
- schema.add(t.third);
- } else {
- schema.add(v1);
- }
+ LinkedHashSet<LogicalVariable> outVarSet = new LinkedHashSet<>();
+ for (int i = 0, ln = inputs.size(); i < ln; i++) {
+ for (LogicalVariable inVar : inputs.get(i).getValue().getSchema()) {
+ LogicalVariable outVar = findOutputVar(inVar, i);
+ outVarSet.add(outVar != null ? outVar : inVar);
}
}
- for (LogicalVariable v2 : inputs.get(1).getValue().getSchema()) {
- for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> t : varMap) {
- if (t.second.equals(v2)) {
- schema.add(t.third);
- } else {
- schema.add(v2);
- }
+ schema = new ArrayList<>(outVarSet);
+ }
+
+ private LogicalVariable findOutputVar(LogicalVariable inputVar, int inputIdx) {
+ for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> t : varMap) {
+ LogicalVariable testVar;
+ switch (inputIdx) {
+ case 0:
+ testVar = t.first;
+ break;
+ case 1:
+ testVar = t.second;
+ break;
+ default:
+ throw new IllegalArgumentException(String.valueOf(inputIdx));
+ }
+ if (inputVar.equals(testVar)) {
+ return t.third;
}
}
+ return null;
}
@Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java
index 54cbcd0..f68638d 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java
@@ -244,6 +244,8 @@
@Override
public Long visitDistinctOperator(DistinctOperator op, Void arg) throws AlgebricksException {
+ // DISTINCT cannot reduce cardinality from ONE to ZERO_OR_ONE, or from ONE_GROUP to ZERO_OR_ONE_GROUP
+ // therefore we don't need to call adjustCardinalityForTupleReductionOperator() here.
return op.getInputs().get(0).getValue().accept(this, arg);
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
index 7b3f53a..e50ea65 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
@@ -460,12 +460,17 @@
return Boolean.FALSE;
}
DataSourceScanOperator argScan = (DataSourceScanOperator) arg;
- if (!argScan.getDataSource().toString().equals(op.getDataSource().toString())) {
+ boolean isomorphic = op.getDataSource().getId().equals(argScan.getDataSource().getId())
+ && op.getOutputLimit() == argScan.getOutputLimit();
+ if (!isomorphic) {
return Boolean.FALSE;
}
DataSourceScanOperator scanOpArg = (DataSourceScanOperator) copyAndSubstituteVar(op, arg);
- boolean isomorphic = VariableUtilities.varListEqualUnordered(op.getVariables(), scanOpArg.getVariables())
- && op.getDataSource().toString().equals(scanOpArg.getDataSource().toString());
+ ILogicalExpression opCondition = op.getSelectCondition() != null ? op.getSelectCondition().getValue() : null;
+ ILogicalExpression argCondition =
+ scanOpArg.getSelectCondition() != null ? scanOpArg.getSelectCondition().getValue() : null;
+ isomorphic = VariableUtilities.varListEqualUnordered(op.getVariables(), scanOpArg.getVariables())
+ && Objects.equals(opCondition, argCondition);
return isomorphic;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractWindowPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractWindowPOperator.java
index 0b3a79c..560435e 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractWindowPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractWindowPOperator.java
@@ -102,8 +102,8 @@
OrderColumn oc = orderColumns.get(oIdx);
LogicalVariable ocVar = oc.getColumn();
if (!pcVars.remove(ocVar) && containsAny(orderColumns, oIdx + 1, pcVars)) {
- throw new AlgebricksException(ErrorCode.HYRACKS, ErrorCode.UNSUPPORTED_WINDOW_SPEC,
- op.getSourceLocation(), String.valueOf(partitionColumns), String.valueOf(orderColumns));
+ throw AlgebricksException.create(ErrorCode.UNSUPPORTED_WINDOW_SPEC, op.getSourceLocation(),
+ String.valueOf(partitionColumns), String.valueOf(orderColumns));
}
lopColumns.add(new OrderColumn(oc.getColumn(), oc.getOrder()));
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
index 21ea331..16a2dee 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
@@ -178,7 +178,7 @@
jsonGenerator.writeEndArray();
}
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -201,7 +201,7 @@
}
jsonGenerator.writeEndObject();
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -212,7 +212,7 @@
writeVariablesAndExpressions(op.getVariables(), op.getExpressions(), indent);
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -223,7 +223,7 @@
writeVariablesAndExpressions(op.getVariables(), op.getExpressions(), indent);
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -233,7 +233,7 @@
jsonGenerator.writeStringField(OPERATOR_FIELD, "empty-tuple-source");
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -255,7 +255,7 @@
}
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -269,7 +269,7 @@
}
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -280,7 +280,7 @@
writeStringFieldExpression(CONDITION_FIELD, op.getCondition(), indent);
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -291,7 +291,7 @@
writeStringFieldExpression(CONDITION_FIELD, op.getCondition(), indent);
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -301,7 +301,7 @@
jsonGenerator.writeStringField(OPERATOR_FIELD, "nested-tuple-source");
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -316,7 +316,7 @@
writeArrayFieldOfOrderExprList("order-by-list", op.getOrderExpressions(), indent);
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -327,7 +327,7 @@
writeVariablesAndExpressions(op.getVariables(), op.getExpressions(), indent);
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -341,7 +341,7 @@
}
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -355,7 +355,7 @@
}
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -368,7 +368,7 @@
writeObjectFieldWithExpressions("partitioned-by", op.getKeyExpressions(), indent);
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -379,7 +379,7 @@
writeStringFieldExpression(CONDITION_FIELD, op.getCondition(), indent);
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -393,7 +393,7 @@
}
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -420,7 +420,7 @@
jsonGenerator.writeEndArray();
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -438,7 +438,7 @@
}
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -461,7 +461,7 @@
writeSelectLimitInformation(op.getSelectCondition(), op.getOutputLimit(), indent);
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -490,7 +490,7 @@
writeSelectLimitInformation(op.getSelectCondition(), op.getOutputLimit(), indent);
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -505,7 +505,7 @@
}
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -515,7 +515,7 @@
jsonGenerator.writeStringField(OPERATOR_FIELD, "exchange");
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -533,7 +533,7 @@
}
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -543,7 +543,7 @@
jsonGenerator.writeStringField(OPERATOR_FIELD, "replicate");
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -554,7 +554,7 @@
writeStringFieldExpression(EXPRESSION_FIELD, op.getBranchingExpression(), indent);
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -564,7 +564,7 @@
jsonGenerator.writeStringField(OPERATOR_FIELD, "materialize");
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -592,7 +592,7 @@
}
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -616,7 +616,7 @@
}
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -627,7 +627,7 @@
writeVariablesAndExpressions(op.getTokenizeVars(), op.getSecondaryKeyExpressions(), indent);
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -638,7 +638,7 @@
writeStringFieldExpression(EXPRESSION_FIELD, op.getSideDataExpression(), indent);
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -648,7 +648,7 @@
jsonGenerator.writeStringField(OPERATOR_FIELD, "sink");
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -658,7 +658,7 @@
jsonGenerator.writeStringField(OPERATOR_FIELD, op.toString());
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -720,7 +720,7 @@
}
return null;
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -735,7 +735,7 @@
jsonGenerator.writeEndArray();
idCounter.previousPrefix();
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -753,7 +753,7 @@
}
writeArrayFieldOfExpression(EXPRESSIONS_FIELD, op.getExpressionRef(), indent);
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -768,7 +768,7 @@
writeArrayFieldOfExpression(EXPRESSIONS_FIELD, op.getExpressionRef(), indent);
writeFilterInformation(op.getMinFilterVars(), op.getMaxFilterVars());
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -786,7 +786,7 @@
jsonGenerator.writeEndObject();
}
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
@@ -926,7 +926,7 @@
try {
jsonGenerator.flush();
} catch (IOException e) {
- throw new AlgebricksException(e, ErrorCode.ERROR_PRINTING_PLAN);
+ throw AlgebricksException.create(ErrorCode.ERROR_PRINTING_PLAN, e, String.valueOf(e));
}
}
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java
index 3f5012a..b8fe5c7 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorManipulationUtil.java
@@ -53,6 +53,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
import org.apache.hyracks.algebricks.core.algebra.plan.ALogicalPlanImpl;
import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
public class OperatorManipulationUtil {
@@ -461,4 +462,15 @@
}
return -1;
}
+
+ public static List<Mutable<ILogicalExpression>> createVariableReferences(List<LogicalVariable> varList,
+ SourceLocation sourceLoc) {
+ List<Mutable<ILogicalExpression>> varRefs = new ArrayList<>(varList.size());
+ for (LogicalVariable var : varList) {
+ VariableReferenceExpression varRef = new VariableReferenceExpression(var);
+ varRef.setSourceLocation(sourceLoc);
+ varRefs.add(new MutableObject<>(varRef));
+ }
+ return varRefs;
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
index 15bb54b..7630a13 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
@@ -18,6 +18,7 @@
*/
package org.apache.hyracks.algebricks.core.config;
+import org.apache.hyracks.util.StorageUtil;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -26,5 +27,7 @@
public static final String ALGEBRICKS_LOGGER_NAME = "org.apache.hyracks.algebricks";
public static final Logger ALGEBRICKS_LOGGER = LogManager.getLogger(ALGEBRICKS_LOGGER_NAME);
public static final int SORT_SAMPLES = 100;
+ public static final int EXTERNAL_SCAN_BUFFER_SIZE =
+ StorageUtil.getIntSizeInBytes(4, StorageUtil.StorageUnit.KILOBYTE);
public static final boolean SORT_PARALLEL = true;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
index ae333e2..3c03b18 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/JobGenContext.java
@@ -34,6 +34,7 @@
import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
import org.apache.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
import org.apache.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
@@ -45,6 +46,7 @@
import org.apache.hyracks.algebricks.data.ITypeTraitProvider;
import org.apache.hyracks.api.dataflow.value.IMissingWriterFactory;
import org.apache.hyracks.api.dataflow.value.IPredicateEvaluatorFactoryProvider;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
public class JobGenContext {
private final IOperatorSchema outerFlowSchema;
@@ -67,9 +69,11 @@
private final IPartialAggregationTypeComputer partialAggregationTypeComputer;
private final IPredicateEvaluatorFactoryProvider predEvaluatorFactoryProvider;
private final int frameSize;
+ private final PhysicalOptimizationConfig physicalOptimizationConfig;
private AlgebricksAbsolutePartitionConstraint clusterLocations;
private int varCounter;
private final ITypingContext typingContext;
+ private final IWarningCollector warningCollector;
private final long maxWarnings;
public JobGenContext(IOperatorSchema outerFlowSchema, IMetadataProvider<?, ?> metadataProvider, Object appContext,
@@ -85,7 +89,8 @@
ITypingContext typingContext, IExpressionEvalSizeComputer expressionEvalSizeComputer,
IPartialAggregationTypeComputer partialAggregationTypeComputer,
IPredicateEvaluatorFactoryProvider predEvaluatorFactoryProvider, int frameSize,
- AlgebricksAbsolutePartitionConstraint clusterLocations, long maxWarnings) {
+ AlgebricksAbsolutePartitionConstraint clusterLocations, IWarningCollector warningCollector,
+ long maxWarnings, PhysicalOptimizationConfig physicalOptimizationConfig) {
this.outerFlowSchema = outerFlowSchema;
this.metadataProvider = metadataProvider;
this.appContext = appContext;
@@ -107,7 +112,9 @@
this.partialAggregationTypeComputer = partialAggregationTypeComputer;
this.predEvaluatorFactoryProvider = predEvaluatorFactoryProvider;
this.frameSize = frameSize;
+ this.physicalOptimizationConfig = physicalOptimizationConfig;
this.varCounter = 0;
+ this.warningCollector = warningCollector;
this.maxWarnings = maxWarnings;
}
@@ -209,7 +216,15 @@
return typingContext.getOutputTypeEnvironment(op);
}
+ public IWarningCollector getWarningCollector() {
+ return warningCollector;
+ }
+
public long getMaxWarnings() {
return maxWarnings;
}
+
+ public PhysicalOptimizationConfig getPhysicalOptimizationConfig() {
+ return physicalOptimizationConfig;
+ }
}
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
index 598497c..93f5d74 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
@@ -39,6 +39,7 @@
private static final String DEFAULT_IN_MEM_HASH_JOIN_TABLE_SIZE = "DEFAULT_IN_MEM_HASH_JOIN_TABLE_SIZE";
private static final String SORT_PARALLEL = "SORT_PARALLEL";
private static final String SORT_SAMPLES = "SORT_SAMPLES";
+ private static final String EXTERNAL_SCAN_BUFFER_SIZE = "EXTERNAL_SCAN_BUFFER_SIZE";
private Properties properties = new Properties();
@@ -172,6 +173,14 @@
setInt(SORT_SAMPLES, sortSamples);
}
+ public int getExternalScanBufferSize() {
+ return getInt(EXTERNAL_SCAN_BUFFER_SIZE, AlgebricksConfig.EXTERNAL_SCAN_BUFFER_SIZE);
+ }
+
+ public void setExternalScanBufferSize(int bufferSize) {
+ setInt(EXTERNAL_SCAN_BUFFER_SIZE, bufferSize);
+ }
+
private void setInt(String property, int value) {
properties.setProperty(property, Integer.toString(value));
}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractIntroduceGroupByCombinerRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractIntroduceGroupByCombinerRule.java
index d9746b2..b917ce1 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractIntroduceGroupByCombinerRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/AbstractIntroduceGroupByCombinerRule.java
@@ -87,7 +87,7 @@
if (!newGbyLiveVars.contains(usedVar)) {
// Let the left-hand side of gbyOp's decoration expressions populated through the combiner group-by without
// any intermediate assignment.
- newGbyOp.addDecorExpression(null, p.second.getValue());
+ newGbyOp.addDecorExpression(null, p.second.getValue().cloneExpression());
newGbyLiveVars.add(usedVar);
}
}
@@ -171,6 +171,9 @@
// Nothing is pushed.
if (bi.modifyGbyMap.isEmpty()) {
+ // Reset the group-by operator with original nested plans.
+ gbyNestedPlans.clear();
+ gbyNestedPlans.addAll(backupNestedPlans);
return null;
}
@@ -188,6 +191,9 @@
LogicalVariable v2 = newOpGbyList.get(i);
if (v != v2) {
// cannot linearize
+ // Reset the group-by operator with original nested plans.
+ gbyNestedPlans.clear();
+ gbyNestedPlans.addAll(backupNestedPlans);
return null;
}
} else {
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
index 706028b..9af21f5 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
@@ -698,9 +698,8 @@
// these two exchange ops are needed so that the parents of replicate stay the same during later optimizations.
// This is because replicate operator has references to its parents. If any later optimizations add new parents,
// then replicate would still point to the old ones.
- MutableObject<ILogicalOperator> replicateOpRef = new MutableObject<>(replicateOp);
- ExchangeOperator exchToLocalAgg = createOneToOneExchangeOp(replicateOpRef, ctx);
- ExchangeOperator exchToForward = createOneToOneExchangeOp(replicateOpRef, ctx);
+ ExchangeOperator exchToLocalAgg = createOneToOneExchangeOp(new MutableObject<>(replicateOp), ctx);
+ ExchangeOperator exchToForward = createOneToOneExchangeOp(new MutableObject<>(replicateOp), ctx);
MutableObject<ILogicalOperator> exchToLocalAggRef = new MutableObject<>(exchToLocalAgg);
MutableObject<ILogicalOperator> exchToForwardRef = new MutableObject<>(exchToForward);
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java
index b8dd24f..9420498 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonExpressionsRule.java
@@ -19,6 +19,7 @@
package org.apache.hyracks.algebricks.rewriter.rules;
import java.util.ArrayList;
+import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
@@ -82,6 +83,9 @@
private final Map<ILogicalExpression, ExprEquivalenceClass> exprEqClassMap =
new HashMap<ILogicalExpression, ExprEquivalenceClass>();
+ private final List<LogicalVariable> tmpLiveVars = new ArrayList<>();
+ private final List<LogicalVariable> tmpProducedVars = new ArrayList<>();
+
// Set of operators for which common subexpression elimination should not be performed.
private static final Set<LogicalOperatorTag> ignoreOps = new HashSet<LogicalOperatorTag>(6);
@@ -123,6 +127,31 @@
exprEqClass.setVariable(lhs);
}
+ // remove equivalence classes that supply given vars
+ private void pruneEquivalenceClassMap(Collection<LogicalVariable> pruneVars) throws AlgebricksException {
+ for (Iterator<Map.Entry<ILogicalExpression, ExprEquivalenceClass>> i = exprEqClassMap.entrySet().iterator(); i
+ .hasNext();) {
+ Map.Entry<ILogicalExpression, ExprEquivalenceClass> me = i.next();
+ ExprEquivalenceClass eqClass = me.getValue();
+ boolean eqClassProvidesPruneVar = false;
+ if (eqClass.variableIsSet() && pruneVars.contains(eqClass.getVariable())) {
+ eqClassProvidesPruneVar = true;
+ } else {
+ tmpProducedVars.clear();
+ VariableUtilities.getProducedVariables(eqClass.getFirstOperator(), tmpProducedVars);
+ for (LogicalVariable producedVar : tmpProducedVars) {
+ if (pruneVars.contains(producedVar)) {
+ eqClassProvidesPruneVar = true;
+ break;
+ }
+ }
+ }
+ if (eqClassProvidesPruneVar) {
+ i.remove();
+ }
+ }
+ }
+
private boolean removeCommonExpressions(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
throws AlgebricksException {
AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
@@ -187,6 +216,12 @@
// Update equivalence class map with original assign expression.
updateEquivalenceClassMap(lhs, exprRef, originalAssignExprs.get(i), op);
}
+ } else if (op.getOperatorTag() == LogicalOperatorTag.LEFTOUTERJOIN) {
+ // remove equivalence classes that provide vars that are live on the right branch of a left outer join
+ ILogicalOperator rightBranchOp = op.getInputs().get(1).getValue();
+ tmpLiveVars.clear();
+ VariableUtilities.getLiveVariables(rightBranchOp, tmpLiveVars);
+ pruneEquivalenceClassMap(tmpLiveVars);
}
// TODO: For now do not perform replacement in nested plans
@@ -256,6 +291,7 @@
}
} else {
if (expr.isFunctional() && assignCommonExpression(exprEqClass, expr)) {
+ modified = true;
//re-obtain the live vars after rewriting in the method called in the if condition
Set<LogicalVariable> liveVars = new HashSet<LogicalVariable>();
VariableUtilities.getLiveVariables(op, liveVars);
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
index 176ab7a..3effcc8 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
@@ -35,7 +35,6 @@
import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractReplicateOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
@@ -48,6 +47,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.physical.OneToOneExchangePOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.ReplicatePOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.StreamProjectPOperator;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
import org.apache.hyracks.api.exceptions.SourceLocation;
@@ -172,7 +172,6 @@
ReplicateOperator rop = new ReplicateOperator(group.size(), materializationFlags);
rop.setSourceLocation(candidateSourceLoc);
rop.setPhysicalOperator(new ReplicatePOperator());
- Mutable<ILogicalOperator> ropRef = new MutableObject<ILogicalOperator>(rop);
AbstractLogicalOperator aopCandidate = (AbstractLogicalOperator) candidate.getValue();
List<Mutable<ILogicalOperator>> originalCandidateParents = childrenToParents.get(candidate);
@@ -194,14 +193,14 @@
AbstractLogicalOperator parent = (AbstractLogicalOperator) parentRef.getValue();
int index = parent.getInputs().indexOf(candidate);
if (parent.getOperatorTag() == LogicalOperatorTag.EXCHANGE) {
- parent.getInputs().set(index, ropRef);
+ parent.getInputs().set(index, new MutableObject<>(rop));
rop.getOutputs().add(parentRef);
} else {
AbstractLogicalOperator exchange = new ExchangeOperator();
exchange.setPhysicalOperator(new OneToOneExchangePOperator());
exchange.setExecutionMode(rop.getExecutionMode());
MutableObject<ILogicalOperator> exchangeRef = new MutableObject<ILogicalOperator>(exchange);
- exchange.getInputs().add(ropRef);
+ exchange.getInputs().add(new MutableObject<>(rop));
rop.getOutputs().add(exchangeRef);
context.computeAndSetTypeEnvironmentForOperator(exchange);
parent.getInputs().set(index, exchangeRef);
@@ -210,12 +209,6 @@
}
List<LogicalVariable> liveVarsNew = new ArrayList<LogicalVariable>();
VariableUtilities.getLiveVariables(candidate.getValue(), liveVarsNew);
- ArrayList<Mutable<ILogicalExpression>> assignExprs = new ArrayList<Mutable<ILogicalExpression>>();
- for (LogicalVariable liveVar : liveVarsNew) {
- VariableReferenceExpression liveVarRef = new VariableReferenceExpression(liveVar);
- liveVarRef.setSourceLocation(candidateSourceLoc);
- assignExprs.add(new MutableObject<ILogicalExpression>(liveVarRef));
- }
for (Mutable<ILogicalOperator> ref : group) {
if (ref.equals(candidate)) {
continue;
@@ -230,6 +223,8 @@
SourceLocation refSourceLoc = ref.getValue().getSourceLocation();
+ List<Mutable<ILogicalExpression>> assignExprs =
+ OperatorManipulationUtil.createVariableReferences(liveVarsNew, candidateSourceLoc);
AbstractLogicalOperator assignOperator = new AssignOperator(liveVars, assignExprs);
assignOperator.setSourceLocation(refSourceLoc);
assignOperator.setExecutionMode(rop.getExecutionMode());
@@ -241,7 +236,7 @@
AbstractLogicalOperator exchOp = new ExchangeOperator();
exchOp.setPhysicalOperator(new OneToOneExchangePOperator());
exchOp.setExecutionMode(rop.getExecutionMode());
- exchOp.getInputs().add(ropRef);
+ exchOp.getInputs().add(new MutableObject<>(rop));
MutableObject<ILogicalOperator> exchOpRef = new MutableObject<ILogicalOperator>(exchOp);
rop.getOutputs().add(exchOpRef);
assignOperator.getInputs().add(exchOpRef);
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IntroduceProjectsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IntroduceProjectsRule.java
index a724014..af67be2 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IntroduceProjectsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/IntroduceProjectsRule.java
@@ -177,6 +177,7 @@
if (liveVars.size() == projectVarsTemp.size() && liveVars.containsAll(projectVarsTemp)) {
// The existing project has become useless. Remove it.
parentOp.getInputs().get(parentInputIndex).setValue(op.getInputs().get(0).getValue());
+ modified = true;
}
}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java
index d0cd006..5f66c2f 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/PushSelectIntoJoinRule.java
@@ -150,18 +150,24 @@
if (!intersectsBranch[0] && !intersectsBranch[1]) {
return false;
}
+ boolean planChanged;
if (needToPushOps) {
//We should push independent ops into the first branch that the selection depends on
- if (intersectsBranch[0]) {
- pushOps(pushedOnEither, joinBranchLeftRef, context);
- } else {
- pushOps(pushedOnEither, joinBranchRightRef, context);
- }
- pushOps(pushedOnLeft, joinBranchLeftRef, context);
- pushOps(pushedOnRight, joinBranchRightRef, context);
+ planChanged =
+ pushOps(pushedOnEither, intersectsBranch[0] ? joinBranchLeftRef : joinBranchRightRef, context);
+ planChanged |= pushOps(pushedOnLeft, joinBranchLeftRef, context);
+ planChanged |= pushOps(pushedOnRight, joinBranchRightRef, context);
+ } else {
+ planChanged = false;
}
if (intersectsAllBranches) {
- addCondToJoin(select, join, context);
+ // add condition to the join condition only if we have IJ
+ if (isLoj) {
+ notPushedStack.addFirst(select);
+ } else {
+ addCondToJoin(select, join, context);
+ planChanged = true;
+ }
} else { // push down
Iterator<Mutable<ILogicalOperator>> branchIter = join.getInputs().iterator();
ILogicalExpression selectCondition = select.getCondition().getValue();
@@ -172,20 +178,20 @@
if (!inter) {
continue;
}
-
- // if a left outer join, if the select condition is not-missing filtering,
- // we rewrite left outer join
- // to inner join for this case.
- if (j > 0 && isLoj && containsNotMissingFiltering(selectCondition)) {
- lojToInner = true;
- }
- if ((j > 0 && isLoj) && containsMissingFiltering(selectCondition)) {
- // Select "is-not-missing($$var)" cannot be pushed in the right branch of a LOJ;
+ if (j > 0 && isLoj) {
+ // if a LOJ and the select condition is not-missing filtering,
+ // we rewrite LOJ to IJ for this case.
+ if (containsNotMissingFiltering(selectCondition)) {
+ lojToInner = true;
+ }
+ // Do not push conditions into the right branch of a LOJ;
notPushedStack.addFirst(select);
} else {
- // Conditions for the left branch can always be pushed.
- // Other conditions can be pushed to the right branch of a LOJ.
+ // Conditions for the left branch for IJ/LOJ or
+ // for the right branch of IJ can always be pushed into that branch.
+ // We don't push conditions into the right branch of LOJ at this point.
copySelectToBranch(select, branch, context);
+ planChanged = true;
}
}
if (lojToInner) {
@@ -194,23 +200,47 @@
innerJoin.getInputs().addAll(join.getInputs());
join = innerJoin;
context.computeAndSetTypeEnvironmentForOperator(join);
+ planChanged = true;
}
}
- ILogicalOperator top = join;
- for (ILogicalOperator npOp : notPushedStack) {
- List<Mutable<ILogicalOperator>> npInpList = npOp.getInputs();
- npInpList.clear();
- npInpList.add(new MutableObject<ILogicalOperator>(top));
- context.computeAndSetTypeEnvironmentForOperator(npOp);
- top = npOp;
- }
- opRef.setValue(top);
- return true;
+ planChanged |= applyNonPushed(opRef, notPushedStack, join, context);
+ return planChanged;
}
- private void pushOps(List<ILogicalOperator> opList, Mutable<ILogicalOperator> joinBranch,
+ private boolean applyNonPushed(Mutable<ILogicalOperator> opRef, LinkedList<ILogicalOperator> notPushedStack,
+ ILogicalOperator top, IOptimizationContext context) throws AlgebricksException {
+ switch (notPushedStack.size()) {
+ case 0:
+ if (opRef.getValue() == top) {
+ return false;
+ }
+ opRef.setValue(top);
+ return true;
+ case 1:
+ ILogicalOperator notPushedOp = notPushedStack.peek();
+ if (opRef.getValue() == notPushedOp && opRef.getValue().getInputs().get(0).getValue() == top) {
+ return false;
+ }
+ // fall thru to 'default'
+ default:
+ for (ILogicalOperator npOp : notPushedStack) {
+ List<Mutable<ILogicalOperator>> npInpList = npOp.getInputs();
+ npInpList.clear();
+ npInpList.add(new MutableObject<>(top));
+ context.computeAndSetTypeEnvironmentForOperator(npOp);
+ top = npOp;
+ }
+ opRef.setValue(top);
+ return true;
+ }
+ }
+
+ private boolean pushOps(List<ILogicalOperator> opList, Mutable<ILogicalOperator> joinBranch,
IOptimizationContext context) throws AlgebricksException {
+ if (opList.isEmpty()) {
+ return false;
+ }
ILogicalOperator topOp = joinBranch.getValue();
ListIterator<ILogicalOperator> iter = opList.listIterator(opList.size());
while (iter.hasPrevious()) {
@@ -222,6 +252,7 @@
context.computeAndSetTypeEnvironmentForOperator(op);
}
joinBranch.setValue(topOp);
+ return true;
}
private static void addCondToJoin(SelectOperator select, AbstractBinaryJoinOperator join,
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantVariablesRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantVariablesRule.java
index 5386193..3760be5 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantVariablesRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/RemoveRedundantVariablesRule.java
@@ -36,7 +36,6 @@
import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
@@ -69,26 +68,33 @@
public class RemoveRedundantVariablesRule implements IAlgebraicRewriteRule {
private final VariableSubstitutionVisitor substVisitor = new VariableSubstitutionVisitor();
- private final Map<LogicalVariable, List<LogicalVariable>> equivalentVarsMap =
- new HashMap<LogicalVariable, List<LogicalVariable>>();
+
+ private final Map<LogicalVariable, List<LogicalVariable>> equivalentVarsMap = new HashMap<>();
@Override
public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
if (context.checkIfInDontApplySet(this, opRef.getValue())) {
return false;
}
- boolean modified = removeRedundantVariables(opRef, context);
- if (modified) {
- context.computeAndSetTypeEnvironmentForOperator(opRef.getValue());
- }
- return modified;
+ clear();
+ return removeRedundantVariables(opRef, true, context);
+ }
+
+ private void clear() {
+ equivalentVarsMap.clear();
}
private void updateEquivalenceClassMap(LogicalVariable lhs, LogicalVariable rhs) {
List<LogicalVariable> equivalentVars = equivalentVarsMap.get(rhs);
if (equivalentVars == null) {
- equivalentVars = new ArrayList<LogicalVariable>();
+ equivalentVars = new ArrayList<>();
// The first element in the list is the bottom-most representative which will replace all equivalent vars.
equivalentVars.add(rhs);
equivalentVars.add(lhs);
@@ -97,12 +103,32 @@
equivalentVarsMap.put(lhs, equivalentVars);
}
- private boolean removeRedundantVariables(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
- throws AlgebricksException {
+ private LogicalVariable findEquivalentRepresentativeVar(LogicalVariable var) {
+ List<LogicalVariable> equivalentVars = equivalentVarsMap.get(var);
+ if (equivalentVars == null) {
+ return null;
+ }
+ LogicalVariable representativeVar = equivalentVars.get(0);
+ return var.equals(representativeVar) ? null : representativeVar;
+ }
+
+ private boolean removeRedundantVariables(Mutable<ILogicalOperator> opRef, boolean first,
+ IOptimizationContext context) throws AlgebricksException {
AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (!first) {
+ context.addToDontApplySet(this, op);
+ }
+
LogicalOperatorTag opTag = op.getOperatorTag();
boolean modified = false;
+ // Recurse into children.
+ for (Mutable<ILogicalOperator> inputOpRef : op.getInputs()) {
+ if (removeRedundantVariables(inputOpRef, false, context)) {
+ modified = true;
+ }
+ }
+
// Update equivalence class map.
if (opTag == LogicalOperatorTag.ASSIGN) {
AssignOperator assignOp = (AssignOperator) op;
@@ -142,7 +168,7 @@
AbstractOperatorWithNestedPlans opWithNestedPlan = (AbstractOperatorWithNestedPlans) op;
for (ILogicalPlan nestedPlan : opWithNestedPlan.getNestedPlans()) {
for (Mutable<ILogicalOperator> rootRef : nestedPlan.getRoots()) {
- if (removeRedundantVariables(rootRef, context)) {
+ if (removeRedundantVariables(rootRef, false, context)) {
modified = true;
}
}
@@ -158,14 +184,8 @@
if (modified) {
context.computeAndSetTypeEnvironmentForOperator(op);
- context.addToDontApplySet(this, op);
}
- // Clears the equivalent variable map if the current operator is the root operator
- // in the query plan.
- if (opTag == LogicalOperatorTag.DISTRIBUTE_RESULT || opTag == LogicalOperatorTag.SINK) {
- equivalentVarsMap.clear();
- }
return modified;
}
@@ -227,38 +247,34 @@
* We cannot use the VariableSubstitutionVisitor here because the project ops
* maintain their variables as a list and not as expressions.
*/
- private boolean replaceProjectVars(ProjectOperator op) throws AlgebricksException {
+ private boolean replaceProjectVars(ProjectOperator op) {
List<LogicalVariable> vars = op.getVariables();
int size = vars.size();
boolean modified = false;
for (int i = 0; i < size; i++) {
LogicalVariable var = vars.get(i);
- List<LogicalVariable> equivalentVars = equivalentVarsMap.get(var);
- if (equivalentVars == null) {
- continue;
- }
- // Replace with equivalence class representative.
- LogicalVariable representative = equivalentVars.get(0);
- if (representative != var) {
- vars.set(i, equivalentVars.get(0));
+ LogicalVariable representativeVar = findEquivalentRepresentativeVar(var);
+ if (representativeVar != null) {
+ // Replace with equivalence class representative.
+ vars.set(i, representativeVar);
modified = true;
}
}
return modified;
}
- private boolean replaceUnionAllVars(UnionAllOperator op) throws AlgebricksException {
+ private boolean replaceUnionAllVars(UnionAllOperator op) {
boolean modified = false;
for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> varMapping : op.getVariableMappings()) {
- List<LogicalVariable> firstEquivalentVars = equivalentVarsMap.get(varMapping.first);
- List<LogicalVariable> secondEquivalentVars = equivalentVarsMap.get(varMapping.second);
// Replace variables with their representative.
- if (firstEquivalentVars != null) {
- varMapping.first = firstEquivalentVars.get(0);
+ LogicalVariable firstRepresentativeVar = findEquivalentRepresentativeVar(varMapping.first);
+ if (firstRepresentativeVar != null) {
+ varMapping.first = firstRepresentativeVar;
modified = true;
}
- if (secondEquivalentVars != null) {
- varMapping.second = secondEquivalentVars.get(0);
+ LogicalVariable secondRepresentativeVar = findEquivalentRepresentativeVar(varMapping.second);
+ if (secondRepresentativeVar != null) {
+ varMapping.second = secondRepresentativeVar;
modified = true;
}
}
@@ -269,17 +285,13 @@
@Override
public boolean transform(Mutable<ILogicalExpression> exprRef) {
ILogicalExpression e = exprRef.getValue();
- switch (((AbstractLogicalExpression) e).getExpressionTag()) {
+ switch (e.getExpressionTag()) {
case VARIABLE: {
// Replace variable references with their equivalent representative in the equivalence class map.
VariableReferenceExpression varRefExpr = (VariableReferenceExpression) e;
LogicalVariable var = varRefExpr.getVariableReference();
- List<LogicalVariable> equivalentVars = equivalentVarsMap.get(var);
- if (equivalentVars == null) {
- return false;
- }
- LogicalVariable representative = equivalentVars.get(0);
- if (representative != var) {
+ LogicalVariable representative = findEquivalentRepresentativeVar(var);
+ if (representative != null) {
varRefExpr.setVariable(representative);
return true;
}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
index 42f9aba..7243827 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
@@ -223,8 +223,8 @@
} else if (gby.getSourceLocation() != null) {
IWarningCollector warningCollector = context.getWarningCollector();
if (warningCollector.shouldWarn()) {
- warningCollector.warn(Warning.forHyracks(gby.getSourceLocation(),
- ErrorCode.INAPPLICABLE_HINT, "Group By", "hash"));
+ warningCollector.warn(Warning.of(gby.getSourceLocation(), ErrorCode.INAPPLICABLE_HINT,
+ "Group By", "hash"));
}
}
}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/MoveFreeVariableOperatorOutOfSubplanRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/MoveFreeVariableOperatorOutOfSubplanRule.java
index 94cae74..66ee3a4 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/MoveFreeVariableOperatorOutOfSubplanRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/MoveFreeVariableOperatorOutOfSubplanRule.java
@@ -92,8 +92,7 @@
ILogicalOperator childOp = childOpRef.getValue();
// Try to move operators that only uses free variables out of the subplan.
- if (movableOperator(currentOp.getOperatorTag())
- && independentOperator(currentOp, liveVarsBeforeSubplan)
+ if (movableOperator(currentOp, liveVarsBeforeSubplan)
&& producedVariablesCanbePropagated(currentOp)) {
extractOperator(subplanOp, inputOp, currentOpRef);
inputOp = currentOp;
@@ -109,12 +108,17 @@
return changed;
}
- // Checks whether the current operator is independent of the nested input pipeline in the subplan.
- private boolean independentOperator(ILogicalOperator op, Set<LogicalVariable> liveVarsBeforeSubplan)
+ private boolean movableOperator(ILogicalOperator op, Set<LogicalVariable> liveVarsBeforeSubplan)
throws AlgebricksException {
+ if (!movableOperatorKind(op.getOperatorTag())) {
+ return false;
+ }
+
+ // Checks whether the current operator is independent of the nested input pipeline in the subplan.
Set<LogicalVariable> usedVars = new HashSet<>();
VariableUtilities.getUsedVariables(op, usedVars);
- return liveVarsBeforeSubplan.containsAll(usedVars);
+ boolean independent = liveVarsBeforeSubplan.containsAll(usedVars);
+ return independent && movableIndependentOperator(op, usedVars);
}
// Checks whether there is a variable killing operator in the nested pipeline
@@ -152,7 +156,11 @@
currentOp.getInputs().get(0).setValue(inputOp);
}
- protected boolean movableOperator(LogicalOperatorTag operatorTag) {
- return (operatorTag == LogicalOperatorTag.ASSIGN || operatorTag == LogicalOperatorTag.SUBPLAN);
+ protected boolean movableOperatorKind(LogicalOperatorTag operatorTag) {
+ return operatorTag == LogicalOperatorTag.ASSIGN || operatorTag == LogicalOperatorTag.SUBPLAN;
+ }
+
+ protected boolean movableIndependentOperator(ILogicalOperator op, Set<LogicalVariable> usedVars) {
+ return true;
}
}
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/PushSubplanIntoGroupByRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/PushSubplanIntoGroupByRule.java
index 8e41a15..420bf12 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/PushSubplanIntoGroupByRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/PushSubplanIntoGroupByRule.java
@@ -70,29 +70,30 @@
rootRef = opRef;
invoked = true;
}
- return rewriteForOperator(rootRef, opRef, context);
+ return rewriteForOperator(rootRef, opRef.getValue(), context);
}
// The core rewriting function for an operator.
- private boolean rewriteForOperator(Mutable<ILogicalOperator> rootRef, Mutable<ILogicalOperator> opRef,
+ private boolean rewriteForOperator(Mutable<ILogicalOperator> rootRef, ILogicalOperator parentOperator,
IOptimizationContext context) throws AlgebricksException {
boolean changed = false;
- ILogicalOperator parentOperator = opRef.getValue();
- for (Mutable<ILogicalOperator> ref : parentOperator.getInputs()) {
+ List<Mutable<ILogicalOperator>> parentInputs = parentOperator.getInputs();
+ for (int i = 0, n = parentInputs.size(); i < n; i++) {
+ Mutable<ILogicalOperator> ref = parentInputs.get(i);
ILogicalOperator op = ref.getValue();
// Only processes subplan operator.
- Deque<SubplanOperator> subplans = new ArrayDeque<>();
if (op.getOperatorTag() != LogicalOperatorTag.SUBPLAN) {
// Recursively rewrites the child plan.
- changed |= rewriteForOperator(rootRef, ref, context);
+ changed |= rewriteForOperator(rootRef, op, context);
continue;
}
+ Deque<SubplanOperator> subplans = new ArrayDeque<>();
while (op.getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
SubplanOperator currentSubplan = (SubplanOperator) op;
// Recursively rewrites the pipelines inside a nested subplan.
for (ILogicalPlan subplan : currentSubplan.getNestedPlans()) {
for (Mutable<ILogicalOperator> nestedRootRef : subplan.getRoots()) {
- changed |= rewriteForOperator(nestedRootRef, nestedRootRef, context);
+ changed |= rewriteForOperator(nestedRootRef, nestedRootRef.getValue(), context);
}
}
subplans.addFirst(currentSubplan);
@@ -106,17 +107,17 @@
// Recursively rewrites the pipelines inside a nested subplan.
for (ILogicalPlan subplan : gby.getNestedPlans()) {
for (Mutable<ILogicalOperator> nestedRootRef : subplan.getRoots()) {
- changed |= rewriteForOperator(nestedRootRef, nestedRootRef, context);
+ changed |= rewriteForOperator(nestedRootRef, nestedRootRef.getValue(), context);
}
}
- changed |= pushSubplansIntoGroupBy(rootRef, parentOperator, subplans, gby, context);
+ changed |= pushSubplansIntoGroupBy(rootRef, parentOperator, i, subplans, gby, context);
}
return changed;
}
// Pushes subplans into the group by operator.
private boolean pushSubplansIntoGroupBy(Mutable<ILogicalOperator> currentRootRef, ILogicalOperator parentOperator,
- Deque<SubplanOperator> subplans, GroupByOperator gby, IOptimizationContext context)
+ int parentChildIdx, Deque<SubplanOperator> subplans, GroupByOperator gby, IOptimizationContext context)
throws AlgebricksException {
boolean changed = false;
List<ILogicalPlan> newGbyNestedPlans = new ArrayList<>();
@@ -204,7 +205,7 @@
newGbyNestedPlans.add(new ALogicalPlanImpl(rootOpRef));
upperSubplanRootRefIterator.remove();
- changed |= true;
+ changed = true;
break;
}
}
@@ -212,10 +213,12 @@
if (upperSubplanRootRefs.isEmpty()) {
subplanNestedPlanIterator.remove();
+ changed = true;
}
}
if (subplan.getNestedPlans().isEmpty()) {
subplanOperatorIterator.remove();
+ changed = true;
}
}
@@ -224,11 +227,19 @@
gby.getNestedPlans().addAll(newGbyNestedPlans);
// Connects the group-by operator with its parent operator.
- ILogicalOperator parent = !subplans.isEmpty() ? subplans.getFirst() : parentOperator;
- parent.getInputs().get(0).setValue(gby);
+ ILogicalOperator parent;
+ int childIdx;
+ if (!subplans.isEmpty()) {
+ parent = subplans.getFirst();
+ childIdx = 0;
+ } else {
+ parent = parentOperator;
+ childIdx = parentChildIdx;
+ }
+ parent.getInputs().get(childIdx).setValue(gby);
// Removes unnecessary pipelines inside the group by operator.
- cleanup(currentRootRef.getValue(), gby);
+ changed |= cleanup(currentRootRef.getValue(), gby);
// Computes type environments.
context.computeAndSetTypeEnvironmentForOperator(gby);
@@ -245,7 +256,8 @@
* the group-by operator.
* @throws AlgebricksException
*/
- private void cleanup(ILogicalOperator rootOp, GroupByOperator gby) throws AlgebricksException {
+ private boolean cleanup(ILogicalOperator rootOp, GroupByOperator gby) throws AlgebricksException {
+ boolean changed = false;
Set<LogicalVariable> freeVars = new HashSet<>();
OperatorPropertiesUtil.getFreeVariablesInPath(rootOp, gby, freeVars);
Iterator<ILogicalPlan> nestedPlanIterator = gby.getNestedPlans().iterator();
@@ -259,16 +271,20 @@
if (!freeVars.contains(aggOp.getVariables().get(varIndex))) {
aggOp.getVariables().remove(varIndex);
aggOp.getExpressions().remove(varIndex);
+ changed = true;
}
}
if (aggOp.getVariables().isEmpty()) {
nestRootRefIterator.remove();
+ changed = true;
}
}
if (nestedPlan.getRoots().isEmpty()) {
nestedPlanIterator.remove();
+ changed = true;
}
}
+ return changed;
}
private Mutable<ILogicalOperator> downToNts(Mutable<ILogicalOperator> opRef) {
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/util/JoinUtils.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/util/JoinUtils.java
index 6eede7f..257798b 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/util/JoinUtils.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/util/JoinUtils.java
@@ -224,7 +224,7 @@
if (OperatorPropertiesUtil.isAlwaysTrueCond(conditionExpr) && sourceLoc != null) {
IWarningCollector warningCollector = context.getWarningCollector();
if (warningCollector.shouldWarn()) {
- warningCollector.warn(Warning.forHyracks(sourceLoc, ErrorCode.CROSS_PRODUCT_JOIN));
+ warningCollector.warn(Warning.of(sourceLoc, ErrorCode.CROSS_PRODUCT_JOIN));
}
}
}
diff --git a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
index 3cee12d..6497fdc 100644
--- a/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
+++ b/hyracks-fullstack/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/meta/SubplanRuntimeFactory.java
@@ -191,7 +191,7 @@
@Override
public void flush() throws HyracksDataException {
- writer.flush();
+ appender.flush(writer);
}
/**
diff --git a/hyracks-fullstack/hyracks/hyracks-api/pom.xml b/hyracks-fullstack/hyracks/hyracks-api/pom.xml
index 90fb5ed..cb76cb3 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-api/pom.xml
@@ -66,7 +66,7 @@
</dependency>
<dependency>
<groupId>org.mockito</groupId>
- <artifactId>mockito-all</artifactId>
+ <artifactId>mockito-core</artifactId>
<scope>test</scope>
</dependency>
<dependency>
@@ -94,5 +94,9 @@
<groupId>org.apache.logging.log4j</groupId>
<artifactId>log4j-api</artifactId>
</dependency>
+ <dependency>
+ <groupId>com.google.guava</groupId>
+ <artifactId>guava</artifactId>
+ </dependency>
</dependencies>
</project>
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/channels/IInputChannel.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/channels/IInputChannel.java
index 4deba7b..7baf268 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/channels/IInputChannel.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/channels/IInputChannel.java
@@ -37,4 +37,9 @@
public void open(IHyracksCommonContext ctx) throws HyracksDataException;
public void close() throws HyracksDataException;
+
+ /**
+ * Called when a failure is encountered while reading data
+ */
+ void fail();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
index 38f49dc..6bbc588 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
@@ -18,9 +18,6 @@
*/
package org.apache.hyracks.api.exceptions;
-import java.io.InputStream;
-import java.util.Map;
-
import org.apache.hyracks.api.util.ErrorMessageUtil;
/**
@@ -29,170 +26,174 @@
* 0 --- 9999: runtime errors
* 10000 ---- 19999: compilation errors
*/
-public class ErrorCode {
- private static final String RESOURCE_PATH = "errormsg/en.properties";
- public static final String HYRACKS = "HYR";
-
+public enum ErrorCode implements IError {
// Runtime error codes.
- public static final int INVALID_OPERATOR_OPERATION = 1;
- public static final int ERROR_PROCESSING_TUPLE = 2;
- public static final int FAILURE_ON_NODE = 3;
- public static final int FILE_WITH_ABSOULTE_PATH_NOT_WITHIN_ANY_IO_DEVICE = 4;
- public static final int FULLTEXT_PHRASE_FOUND = 5;
- public static final int JOB_QUEUE_FULL = 6;
- public static final int INVALID_NETWORK_ADDRESS = 7;
- public static final int INVALID_INPUT_PARAMETER = 8;
- public static final int JOB_REQUIREMENTS_EXCEED_CAPACITY = 9;
- public static final int NO_SUCH_NODE = 10;
- public static final int CLASS_LOADING_ISSUE = 11;
- public static final int ILLEGAL_WRITE_AFTER_FLUSH_ATTEMPT = 12;
- public static final int DUPLICATE_IODEVICE = 13;
- public static final int NESTED_IODEVICES = 14;
- public static final int MORE_THAN_ONE_RESULT = 15;
- public static final int RESULT_FAILURE_EXCEPTION = 16;
- public static final int RESULT_FAILURE_NO_EXCEPTION = 17;
- public static final int INCONSISTENT_RESULT_METADATA = 18;
- public static final int CANNOT_DELETE_FILE = 19;
- public static final int NOT_A_JOBID = 20;
- public static final int ERROR_FINDING_DEPLOYED_JOB = 21;
- public static final int DUPLICATE_DEPLOYED_JOB = 22;
- public static final int DEPLOYED_JOB_FAILURE = 23;
- public static final int NO_RESULT_SET = 24;
- public static final int JOB_CANCELED = 25;
- public static final int NODE_FAILED = 26;
- public static final int FILE_IS_NOT_DIRECTORY = 27;
- public static final int CANNOT_READ_FILE = 28;
- public static final int UNIDENTIFIED_IO_ERROR_READING_FILE = 29;
- public static final int FILE_DOES_NOT_EXIST = 30;
- public static final int UNIDENTIFIED_IO_ERROR_DELETING_DIR = 31;
- public static final int RESULT_NO_RECORD = 32;
- public static final int DUPLICATE_KEY = 33;
- public static final int LOAD_NON_EMPTY_INDEX = 34;
- public static final int MODIFY_NOT_SUPPORTED_IN_EXTERNAL_INDEX = 35;
- public static final int FLUSH_NOT_SUPPORTED_IN_EXTERNAL_INDEX = 36;
- public static final int UPDATE_OR_DELETE_NON_EXISTENT_KEY = 37;
- public static final int INDEX_NOT_UPDATABLE = 38;
- public static final int OCCURRENCE_THRESHOLD_PANIC_EXCEPTION = 39;
- public static final int UNKNOWN_INVERTED_INDEX_TYPE = 40;
- public static final int CANNOT_PROPOSE_LINEARIZER_DIFF_DIMENSIONS = 41;
- public static final int CANNOT_PROPOSE_LINEARIZER_FOR_TYPE = 42;
- public static final int RECORD_IS_TOO_LARGE = 43;
- public static final int FAILED_TO_RE_FIND_PARENT = 44;
- public static final int FAILED_TO_FIND_TUPLE = 45;
- public static final int UNSORTED_LOAD_INPUT = 46;
- public static final int OPERATION_EXCEEDED_MAX_RESTARTS = 47;
- public static final int DUPLICATE_LOAD_INPUT = 48;
- public static final int CANNOT_CREATE_ACTIVE_INDEX = 49;
- public static final int CANNOT_ACTIVATE_ACTIVE_INDEX = 50;
- public static final int CANNOT_DEACTIVATE_INACTIVE_INDEX = 51;
- public static final int CANNOT_DESTROY_ACTIVE_INDEX = 52;
- public static final int CANNOT_CLEAR_INACTIVE_INDEX = 53;
- public static final int CANNOT_ALLOCATE_MEMORY_FOR_INACTIVE_INDEX = 54;
- public static final int RESOURCE_DOES_NOT_EXIST = 55;
- public static final int DISK_COMPONENT_SCAN_NOT_ALLOWED_FOR_SECONDARY_INDEX = 56;
- public static final int CANNOT_FIND_MATTER_TUPLE_FOR_ANTI_MATTER_TUPLE = 57;
- public static final int TASK_ABORTED = 58;
- public static final int OPEN_ON_OPEN_WRITER = 59;
- public static final int OPEN_ON_FAILED_WRITER = 60;
- public static final int NEXT_FRAME_ON_FAILED_WRITER = 61;
- public static final int NEXT_FRAME_ON_CLOSED_WRITER = 62;
- public static final int FLUSH_ON_FAILED_WRITER = 63;
- public static final int FLUSH_ON_CLOSED_WRITER = 64;
- public static final int FAIL_ON_FAILED_WRITER = 65;
- public static final int MISSED_FAIL_CALL = 66;
- public static final int CANNOT_CREATE_FILE = 67;
- public static final int NO_MAPPING_FOR_FILE_ID = 68;
- public static final int NO_MAPPING_FOR_FILENAME = 69;
- public static final int CANNOT_GET_NUMBER_OF_ELEMENT_FROM_INACTIVE_FILTER = 70;
- public static final int CANNOT_CREATE_BLOOM_FILTER_BUILDER_FOR_INACTIVE_FILTER = 71;
- public static final int CANNOT_CREATE_BLOOM_FILTER_WITH_NUMBER_OF_PAGES = 72;
- public static final int CANNOT_ADD_TUPLES_TO_DUMMY_BLOOM_FILTER = 73;
- public static final int CANNOT_CREATE_ACTIVE_BLOOM_FILTER = 74;
- public static final int CANNOT_DEACTIVATE_INACTIVE_BLOOM_FILTER = 75;
- public static final int CANNOT_DESTROY_ACTIVE_BLOOM_FILTER = 76;
- public static final int CANNOT_PURGE_ACTIVE_INDEX = 77;
- public static final int CANNOT_PURGE_ACTIVE_BLOOM_FILTER = 78;
- public static final int CANNOT_BULK_LOAD_NON_EMPTY_TREE = 79;
- public static final int CANNOT_CREATE_EXISTING_INDEX = 80;
- public static final int FILE_ALREADY_MAPPED = 81;
- public static final int FILE_ALREADY_EXISTS = 82;
- public static final int NO_INDEX_FOUND_WITH_RESOURCE_ID = 83;
- public static final int FOUND_OVERLAPPING_LSM_FILES = 84;
- public static final int FOUND_MULTIPLE_TRANSACTIONS = 85;
- public static final int UNRECOGNIZED_INDEX_COMPONENT_FILE = 86;
- public static final int UNEQUAL_NUM_FILTERS_TREES = 87;
- public static final int INDEX_NOT_MODIFIABLE = 88;
- public static final int GROUP_BY_MEMORY_BUDGET_EXCEEDS = 89;
- public static final int ILLEGAL_MEMORY_BUDGET = 90;
- public static final int TIMEOUT = 91;
- public static final int JOB_HAS_BEEN_CLEARED_FROM_HISTORY = 92;
- public static final int FAILED_TO_READ_RESULT = 93;
- public static final int CANNOT_READ_CLOSED_FILE = 94;
- public static final int TUPLE_CANNOT_FIT_INTO_EMPTY_FRAME = 95;
- public static final int ILLEGAL_ATTEMPT_TO_ENTER_EMPTY_COMPONENT = 96;
- public static final int ILLEGAL_ATTEMPT_TO_EXIT_EMPTY_COMPONENT = 97;
- public static final int A_FLUSH_OPERATION_HAS_FAILED = 98;
- public static final int A_MERGE_OPERATION_HAS_FAILED = 99;
- public static final int FAILED_TO_SHUTDOWN_EVENT_PROCESSOR = 100;
- public static final int PAGE_DOES_NOT_EXIST_IN_FILE = 101;
- public static final int VBC_ALREADY_OPEN = 102;
- public static final int VBC_ALREADY_CLOSED = 103;
- public static final int INDEX_DOES_NOT_EXIST = 104;
- public static final int CANNOT_DROP_IN_USE_INDEX = 105;
- public static final int CANNOT_DEACTIVATE_PINNED_BLOOM_FILTER = 106;
- public static final int PREDICATE_CANNOT_BE_NULL = 107;
- public static final int FULLTEXT_ONLY_EXECUTABLE_FOR_STRING_OR_LIST = 108;
- public static final int NOT_ENOUGH_BUDGET_FOR_TEXTSEARCH = 109;
- public static final int CANNOT_CONTINUE_TEXT_SEARCH_HYRACKS_TASK_IS_NULL = 110;
- public static final int CANNOT_CONTINUE_TEXT_SEARCH_BUFFER_MANAGER_IS_NULL = 111;
- public static final int CANNOT_ADD_ELEMENT_TO_INVERTED_INDEX_SEARCH_RESULT = 112;
- public static final int UNDEFINED_INVERTED_LIST_MERGE_TYPE = 113;
- public static final int NODE_IS_NOT_ACTIVE = 114;
- public static final int LOCAL_NETWORK_ERROR = 115;
- public static final int ONE_TUPLE_RANGEMAP_EXPECTED = 116;
- public static final int NO_RANGEMAP_PRODUCED = 117;
- public static final int RANGEMAP_NOT_FOUND = 118;
- public static final int UNSUPPORTED_WINDOW_SPEC = 119;
- public static final int EOF = 120;
- public static final int NUMERIC_PROMOTION_ERROR = 121;
- public static final int ERROR_PRINTING_PLAN = 122;
- public static final int INSUFFICIENT_MEMORY = 123;
+ INVALID_OPERATOR_OPERATION(1),
+ ERROR_PROCESSING_TUPLE(2),
+ FAILURE_ON_NODE(3),
+ FILE_WITH_ABSOULTE_PATH_NOT_WITHIN_ANY_IO_DEVICE(4),
+ FULLTEXT_PHRASE_FOUND(5),
+ JOB_QUEUE_FULL(6),
+ INVALID_NETWORK_ADDRESS(7),
+ INVALID_INPUT_PARAMETER(8),
+ JOB_REQUIREMENTS_EXCEED_CAPACITY(9),
+ NO_SUCH_NODE(10),
+ CLASS_LOADING_ISSUE(11),
+ ILLEGAL_WRITE_AFTER_FLUSH_ATTEMPT(12),
+ DUPLICATE_IODEVICE(13),
+ NESTED_IODEVICES(14),
+ MORE_THAN_ONE_RESULT(15),
+ RESULT_FAILURE_EXCEPTION(16),
+ RESULT_FAILURE_NO_EXCEPTION(17),
+ INCONSISTENT_RESULT_METADATA(18),
+ CANNOT_DELETE_FILE(19),
+ NOT_A_JOBID(20),
+ ERROR_FINDING_DEPLOYED_JOB(21),
+ DUPLICATE_DEPLOYED_JOB(22),
+ DEPLOYED_JOB_FAILURE(23),
+ NO_RESULT_SET(24),
+ JOB_CANCELED(25),
+ NODE_FAILED(26),
+ FILE_IS_NOT_DIRECTORY(27),
+ CANNOT_READ_FILE(28),
+ UNIDENTIFIED_IO_ERROR_READING_FILE(29),
+ FILE_DOES_NOT_EXIST(30),
+ UNIDENTIFIED_IO_ERROR_DELETING_DIR(31),
+ RESULT_NO_RECORD(32),
+ DUPLICATE_KEY(33),
+ LOAD_NON_EMPTY_INDEX(34),
+ MODIFY_NOT_SUPPORTED_IN_EXTERNAL_INDEX(35),
+ FLUSH_NOT_SUPPORTED_IN_EXTERNAL_INDEX(36),
+ UPDATE_OR_DELETE_NON_EXISTENT_KEY(37),
+ INDEX_NOT_UPDATABLE(38),
+ OCCURRENCE_THRESHOLD_PANIC_EXCEPTION(39),
+ UNKNOWN_INVERTED_INDEX_TYPE(40),
+ CANNOT_PROPOSE_LINEARIZER_DIFF_DIMENSIONS(41),
+ CANNOT_PROPOSE_LINEARIZER_FOR_TYPE(42),
+ RECORD_IS_TOO_LARGE(43),
+ FAILED_TO_RE_FIND_PARENT(44),
+ FAILED_TO_FIND_TUPLE(45),
+ UNSORTED_LOAD_INPUT(46),
+ OPERATION_EXCEEDED_MAX_RESTARTS(47),
+ DUPLICATE_LOAD_INPUT(48),
+ CANNOT_CREATE_ACTIVE_INDEX(49),
+ CANNOT_ACTIVATE_ACTIVE_INDEX(50),
+ CANNOT_DEACTIVATE_INACTIVE_INDEX(51),
+ CANNOT_DESTROY_ACTIVE_INDEX(52),
+ CANNOT_CLEAR_INACTIVE_INDEX(53),
+ CANNOT_ALLOCATE_MEMORY_FOR_INACTIVE_INDEX(54),
+ RESOURCE_DOES_NOT_EXIST(55),
+ DISK_COMPONENT_SCAN_NOT_ALLOWED_FOR_SECONDARY_INDEX(56),
+ CANNOT_FIND_MATTER_TUPLE_FOR_ANTI_MATTER_TUPLE(57),
+ TASK_ABORTED(58),
+ OPEN_ON_OPEN_WRITER(59),
+ OPEN_ON_FAILED_WRITER(60),
+ NEXT_FRAME_ON_FAILED_WRITER(61),
+ NEXT_FRAME_ON_CLOSED_WRITER(62),
+ FLUSH_ON_FAILED_WRITER(63),
+ FLUSH_ON_CLOSED_WRITER(64),
+ FAIL_ON_FAILED_WRITER(65),
+ MISSED_FAIL_CALL(66),
+ CANNOT_CREATE_FILE(67),
+ NO_MAPPING_FOR_FILE_ID(68),
+ NO_MAPPING_FOR_FILENAME(69),
+ CANNOT_GET_NUMBER_OF_ELEMENT_FROM_INACTIVE_FILTER(70),
+ CANNOT_CREATE_BLOOM_FILTER_BUILDER_FOR_INACTIVE_FILTER(71),
+ CANNOT_CREATE_BLOOM_FILTER_WITH_NUMBER_OF_PAGES(72),
+ CANNOT_ADD_TUPLES_TO_DUMMY_BLOOM_FILTER(73),
+ CANNOT_CREATE_ACTIVE_BLOOM_FILTER(74),
+ CANNOT_DEACTIVATE_INACTIVE_BLOOM_FILTER(75),
+ CANNOT_DESTROY_ACTIVE_BLOOM_FILTER(76),
+ CANNOT_PURGE_ACTIVE_INDEX(77),
+ CANNOT_PURGE_ACTIVE_BLOOM_FILTER(78),
+ CANNOT_BULK_LOAD_NON_EMPTY_TREE(79),
+ CANNOT_CREATE_EXISTING_INDEX(80),
+ FILE_ALREADY_MAPPED(81),
+ FILE_ALREADY_EXISTS(82),
+ NO_INDEX_FOUND_WITH_RESOURCE_ID(83),
+ FOUND_OVERLAPPING_LSM_FILES(84),
+ FOUND_MULTIPLE_TRANSACTIONS(85),
+ UNRECOGNIZED_INDEX_COMPONENT_FILE(86),
+ UNEQUAL_NUM_FILTERS_TREES(87),
+ INDEX_NOT_MODIFIABLE(88),
+ GROUP_BY_MEMORY_BUDGET_EXCEEDS(89),
+ ILLEGAL_MEMORY_BUDGET(90),
+ TIMEOUT(91),
+ JOB_HAS_BEEN_CLEARED_FROM_HISTORY(92),
+ FAILED_TO_READ_RESULT(93),
+ CANNOT_READ_CLOSED_FILE(94),
+ TUPLE_CANNOT_FIT_INTO_EMPTY_FRAME(95),
+ ILLEGAL_ATTEMPT_TO_ENTER_EMPTY_COMPONENT(96),
+ ILLEGAL_ATTEMPT_TO_EXIT_EMPTY_COMPONENT(97),
+ A_FLUSH_OPERATION_HAS_FAILED(98),
+ A_MERGE_OPERATION_HAS_FAILED(99),
+ FAILED_TO_SHUTDOWN_EVENT_PROCESSOR(100),
+ PAGE_DOES_NOT_EXIST_IN_FILE(101),
+ VBC_ALREADY_OPEN(102),
+ VBC_ALREADY_CLOSED(103),
+ INDEX_DOES_NOT_EXIST(104),
+ CANNOT_DROP_IN_USE_INDEX(105),
+ CANNOT_DEACTIVATE_PINNED_BLOOM_FILTER(106),
+ PREDICATE_CANNOT_BE_NULL(107),
+ FULLTEXT_ONLY_EXECUTABLE_FOR_STRING_OR_LIST(108),
+ NOT_ENOUGH_BUDGET_FOR_TEXTSEARCH(109),
+ CANNOT_CONTINUE_TEXT_SEARCH_HYRACKS_TASK_IS_NULL(110),
+ CANNOT_CONTINUE_TEXT_SEARCH_BUFFER_MANAGER_IS_NULL(111),
+ CANNOT_ADD_ELEMENT_TO_INVERTED_INDEX_SEARCH_RESULT(112),
+ UNDEFINED_INVERTED_LIST_MERGE_TYPE(113),
+ NODE_IS_NOT_ACTIVE(114),
+ LOCAL_NETWORK_ERROR(115),
+ ONE_TUPLE_RANGEMAP_EXPECTED(116),
+ NO_RANGEMAP_PRODUCED(117),
+ RANGEMAP_NOT_FOUND(118),
+ UNSUPPORTED_WINDOW_SPEC(119),
+ EOF(120),
+ NUMERIC_PROMOTION_ERROR(121),
+ ERROR_PRINTING_PLAN(122),
+ INSUFFICIENT_MEMORY(123),
+ PARSING_ERROR(124),
// Compilation error codes.
- public static final int RULECOLLECTION_NOT_INSTANCE_OF_LIST = 10000;
- public static final int CANNOT_COMPOSE_PART_CONSTRAINTS = 10001;
- public static final int PHYS_OPERATOR_NOT_SET = 10002;
- public static final int DESCRIPTOR_GENERATION_ERROR = 10003;
- public static final int EXPR_NOT_NORMALIZED = 10004;
- public static final int OPERATOR_NOT_IMPLEMENTED = 10005;
- public static final int INAPPLICABLE_HINT = 10006;
- public static final int CROSS_PRODUCT_JOIN = 10007;
- public static final int GROUP_ALL_DECOR = 10008;
+ RULECOLLECTION_NOT_INSTANCE_OF_LIST(10000),
+ CANNOT_COMPOSE_PART_CONSTRAINTS(10001),
+ PHYS_OPERATOR_NOT_SET(10002),
+ DESCRIPTOR_GENERATION_ERROR(10003),
+ EXPR_NOT_NORMALIZED(10004),
+ OPERATOR_NOT_IMPLEMENTED(10005),
+ INAPPLICABLE_HINT(10006),
+ CROSS_PRODUCT_JOIN(10007),
+ GROUP_ALL_DECOR(10008);
- private static class Holder {
- private static final Map<Integer, String> errorMessageMap;
+ private static final String RESOURCE_PATH = "errormsg/en.properties";
+ public static final String HYRACKS = "HYR";
+ private final int intValue;
+ private volatile String message;
- static {
- // Loads the map that maps error codes to error message templates.
- try (InputStream resourceStream = ErrorCode.class.getClassLoader().getResourceAsStream(RESOURCE_PATH)) {
- errorMessageMap = ErrorMessageUtil.loadErrorMap(resourceStream);
- } catch (Exception e) {
- throw new IllegalStateException(e);
- }
- }
-
- private Holder() {
- }
+ private ErrorCode(int intValue) {
+ this.intValue = intValue;
}
- private ErrorCode() {
+ @Override
+ public String component() {
+ return HYRACKS;
}
- public static String getErrorMessage(int errorCode) {
- String msg = Holder.errorMessageMap.get(errorCode);
- if (msg == null) {
- throw new IllegalStateException("Undefined error code: " + errorCode);
+ @Override
+ public int intValue() {
+ return intValue;
+ }
+
+ @Override
+ public String errorMessage() {
+ return ErrorMessageMapHolder.get(this);
+ }
+
+ private static class ErrorMessageMapHolder {
+ private static final String[] enumMessages =
+ ErrorMessageUtil.defineMessageEnumOrdinalMap(values(), RESOURCE_PATH);
+
+ private static String get(ErrorCode errorCode) {
+ return enumMessages[errorCode.ordinal()];
}
- return msg;
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java
index 54e4eaf..6deff8d 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksDataException.java
@@ -21,8 +21,6 @@
import java.io.Serializable;
-import org.apache.hyracks.api.util.ErrorMessageUtil;
-
/**
* The main execution time exception type for runtime errors in a hyracks environment
*/
@@ -51,34 +49,25 @@
return new HyracksDataException(cause);
}
- public static HyracksDataException create(int code, SourceLocation sourceLoc, Serializable... params) {
- return new HyracksDataException(ErrorCode.HYRACKS, code, ErrorCode.getErrorMessage(code), null, sourceLoc,
- params);
+ public static HyracksDataException create(ErrorCode code, SourceLocation sourceLoc, Serializable... params) {
+ return new HyracksDataException(code, sourceLoc, params);
}
- public static HyracksDataException create(int code, Serializable... params) {
- return new HyracksDataException(ErrorCode.HYRACKS, code, ErrorCode.getErrorMessage(code), params);
+ public static HyracksDataException create(ErrorCode code, Serializable... params) {
+ return new HyracksDataException(code, params);
}
- public static HyracksDataException create(int code, Throwable cause, SourceLocation sourceLoc,
+ public static HyracksDataException create(ErrorCode code, Throwable cause, SourceLocation sourceLoc,
Serializable... params) {
- return new HyracksDataException(ErrorCode.HYRACKS, code, ErrorCode.getErrorMessage(code), cause, sourceLoc,
- params);
+ return new HyracksDataException(code, cause, sourceLoc, params);
}
- public static HyracksDataException create(int code, Throwable cause, Serializable... params) {
- return new HyracksDataException(ErrorCode.HYRACKS, code, ErrorCode.getErrorMessage(code), cause, params);
+ public static HyracksDataException create(ErrorCode code, Throwable cause, Serializable... params) {
+ return new HyracksDataException(code, cause, null, params);
}
- public HyracksDataException(String component, int errorCode, String message, Throwable cause, String nodeId,
- Serializable... params) {
- super(component, errorCode, message, cause, nodeId, params);
- }
-
- public HyracksDataException(String component, int errorCode, String message, Throwable cause, String nodeId,
- StackTraceElement[] stackTrace, Serializable... params) {
- super(component, errorCode, message, cause, nodeId, params);
- setStackTrace(stackTrace);
+ public static HyracksDataException create(HyracksDataException e, String nodeId) {
+ return new HyracksDataException(e, nodeId);
}
/**
@@ -101,34 +90,24 @@
super(message, cause);
}
- public HyracksDataException(String component, int errorCode, Serializable... params) {
- super(component, errorCode, null, null, null, params);
+ public HyracksDataException(ErrorCode code, Serializable... params) {
+ this(code, null, params);
}
- public HyracksDataException(Throwable cause, int errorCode, Serializable... params) {
- super(ErrorMessageUtil.NONE, errorCode, cause.getMessage(), cause, null, params);
+ public HyracksDataException(ErrorCode code, SourceLocation sourceLoc, Serializable... params) {
+ this(code, null, sourceLoc, params);
}
- public HyracksDataException(String component, int errorCode, String message, Serializable... params) {
- super(component, errorCode, message, null, null, params);
+ public HyracksDataException(ErrorCode code, Throwable cause, SourceLocation sourceLoc, Serializable... params) {
+ super(code, code.component(), code.intValue(), code.errorMessage(), cause, sourceLoc, null, params);
}
- public HyracksDataException(String component, int errorCode, Throwable cause, Serializable... params) {
- super(component, errorCode, cause, params);
+ private HyracksDataException(HyracksDataException hde, String nodeId) {
+ super(hde.getError().orElse(null), hde.getComponent(), hde.getErrorCode(), hde.getMessage(), hde.getCause(),
+ hde.getSourceLocation(), nodeId, hde.getStackTrace(), hde.getParams());
}
- public HyracksDataException(String component, int errorCode, String message, Throwable cause,
- Serializable... params) {
- super(component, errorCode, message, cause, null, params);
- }
-
- public HyracksDataException(String component, int errorCode, String message, Throwable cause,
- SourceLocation sourceLoc, Serializable... params) {
- super(component, errorCode, message, cause, sourceLoc, null, params);
- }
-
- public static HyracksDataException create(HyracksDataException e, String nodeId) {
- return new HyracksDataException(e.getComponent(), e.getErrorCode(), e.getMessage(), e.getCause(), nodeId,
- e.getStackTrace(), e.getParams());
+ protected HyracksDataException(IError error, Throwable cause, SourceLocation sourceLoc, Serializable... params) {
+ super(error, error.component(), error.intValue(), error.errorMessage(), cause, sourceLoc, null, params);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java
index 89e46de..160be7b 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java
@@ -20,7 +20,7 @@
import java.io.IOException;
import java.io.Serializable;
-import java.util.Objects;
+import java.util.Optional;
import org.apache.hyracks.api.util.ErrorMessageUtil;
@@ -28,10 +28,12 @@
private static final long serialVersionUID = 1L;
public static final int UNKNOWN = 0;
+ private static final boolean PARAM_SANITY = true;
private final String component;
private final int errorCode;
private final Serializable[] params;
private final String nodeId;
+ protected transient final IError error;
private SourceLocation sourceLoc;
private transient volatile String msgCache;
@@ -53,22 +55,12 @@
return new HyracksException(cause);
}
- public static HyracksException create(int code, Serializable... params) {
- return new HyracksException(ErrorCode.HYRACKS, code, ErrorCode.getErrorMessage(code), params);
+ public static HyracksException create(ErrorCode code, Serializable... params) {
+ return new HyracksException(code, params);
}
- public static HyracksException create(int code, Throwable cause, Serializable... params) {
- return new HyracksException(ErrorCode.HYRACKS, code, ErrorCode.getErrorMessage(code), cause, params);
- }
-
- public HyracksException(String component, int errorCode, String message, Throwable cause, SourceLocation sourceLoc,
- String nodeId, Serializable... params) {
- super(message, cause);
- this.sourceLoc = sourceLoc;
- this.component = component;
- this.errorCode = errorCode;
- this.nodeId = nodeId;
- this.params = params;
+ public static HyracksException create(ErrorCode code, Throwable cause, Serializable... params) {
+ return new HyracksException(code, cause, params);
}
/**
@@ -76,7 +68,24 @@
*/
@Deprecated
public HyracksException(String message) {
- this(ErrorMessageUtil.NONE, UNKNOWN, message, null, null, (Serializable[]) null);
+ this(message, null);
+ }
+
+ protected HyracksException(IError error, String component, int intCode, String message, Throwable cause,
+ SourceLocation sourceLoc, String nodeId, Serializable... params) {
+ super(message, cause);
+ this.error = error;
+ this.sourceLoc = sourceLoc;
+ this.component = component;
+ this.errorCode = intCode;
+ this.nodeId = nodeId;
+ this.params = params;
+ }
+
+ protected HyracksException(IError errorCode, Throwable cause, SourceLocation sourceLoc, String nodeId,
+ Serializable... params) {
+ this(errorCode, errorCode.component(), errorCode.intValue(), errorCode.errorMessage(), cause, sourceLoc, nodeId,
+ params);
}
/**
@@ -84,7 +93,7 @@
*/
@Deprecated
protected HyracksException(Throwable cause) {
- this(ErrorMessageUtil.NONE, UNKNOWN, ErrorMessageUtil.getCauseMessage(cause), cause, (Serializable[]) null);
+ this(String.valueOf(cause), cause);
}
/**
@@ -92,32 +101,24 @@
*/
@Deprecated
protected HyracksException(String message, Throwable cause) {
- this(ErrorMessageUtil.NONE, UNKNOWN, message, cause, (String) null);
+ this(ErrorMessageUtil.NONE, UNKNOWN, message, cause);
}
- public HyracksException(String component, int errorCode, Serializable... params) {
- this(component, errorCode, null, null, null, params);
+ public HyracksException(ErrorCode errorCode, Throwable cause, Serializable... params) {
+ this(errorCode.component(), errorCode.intValue(), errorCode.errorMessage(), cause, null, params);
}
- public HyracksException(Throwable cause, int errorCode, Serializable... params) {
- this(ErrorMessageUtil.NONE, errorCode, ErrorMessageUtil.getCauseMessage(cause), cause, null, params);
+ public HyracksException(ErrorCode errorCode, Serializable... params) {
+ this(errorCode.component(), errorCode.intValue(), errorCode.errorMessage(), null, null, params);
}
- public HyracksException(String component, int errorCode, String message, Serializable... params) {
- this(component, errorCode, message, null, null, params);
- }
-
- public HyracksException(String component, int errorCode, Throwable cause, Serializable... params) {
- this(component, errorCode, ErrorMessageUtil.getCauseMessage(cause), cause, null, params);
- }
-
- public HyracksException(String component, int errorCode, String message, Throwable cause, Serializable... params) {
+ private HyracksException(String component, int errorCode, String message, Throwable cause, Serializable... params) {
this(component, errorCode, message, cause, null, params);
}
- public HyracksException(String component, int errorCode, String message, Throwable cause, String nodeId,
+ private HyracksException(String component, int errorCode, String message, Throwable cause, String nodeId,
Serializable... params) {
- this(component, errorCode, message, cause, null, nodeId, params);
+ this(null, component, errorCode, message, cause, null, nodeId, params);
}
@Override
@@ -130,7 +131,7 @@
return errorCode;
}
- public Object[] getParams() {
+ public Serializable[] getParams() {
return params;
}
@@ -148,19 +149,25 @@
@Override
public String getMessage() {
- if (msgCache == null) {
- msgCache = ErrorMessageUtil.formatMessage(component, errorCode, super.getMessage(), sourceLoc, params);
+ String message = msgCache;
+ if (message == null) {
+ msgCache = message =
+ ErrorMessageUtil.formatMessage(component, errorCode, super.getMessage(), sourceLoc, params);
}
- return msgCache;
- }
-
- public boolean matches(String component, int errorCode) {
- Objects.requireNonNull(component, "component");
- return component.equals(this.component) && errorCode == this.errorCode;
+ return message;
}
@Override
public String toString() {
return getLocalizedMessage();
}
+
+ @Override
+ public Optional<IError> getError() {
+ return Optional.ofNullable(error);
+ }
+
+ public boolean matches(ErrorCode errorCode) {
+ return component.equals(errorCode.component()) && getErrorCode() == errorCode.intValue();
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/IError.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/IError.java
new file mode 100644
index 0000000..b51bf35
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/IError.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.api.exceptions;
+
+/**
+ * A type-safe error, consisting of a string-based component, integer value, and error message
+ * @since 0.3.5.1
+ */
+public interface IError {
+ String component();
+
+ int intValue();
+
+ String errorMessage();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/IFormattedException.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/IFormattedException.java
index 0f873e1..cdd17bd 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/IFormattedException.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/IFormattedException.java
@@ -18,6 +18,11 @@
*/
package org.apache.hyracks.api.exceptions;
+import java.io.Serializable;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.stream.Stream;
+
public interface IFormattedException {
/**
@@ -40,4 +45,36 @@
* @return the exception message
*/
String getMessage();
+
+ /**
+ * If available, returns the {@link IError} associated with this exception
+ * @return the error instance, othewise {@link Optional#empty()}
+ * @since 0.3.5.1
+ */
+ Optional<IError> getError();
+
+ /**
+ * @return the source location
+ */
+ SourceLocation getSourceLocation();
+
+ /**
+ * @return the parameters to use when formatting
+ */
+ Serializable[] getParams();
+
+ /**
+ * Indicates whether this exception matches the supplied error code
+ */
+ default boolean matches(IError candidate) {
+ Objects.requireNonNull(candidate, "candidate");
+ return getComponent().equals(candidate.component()) && getErrorCode() == candidate.intValue();
+ }
+
+ /**
+ * Indicates whether this exception matches any of the supplied error codes
+ */
+ default boolean matchesAny(IError candidate, IError... otherCandidates) {
+ return matches(candidate) || Stream.of(otherCandidates).anyMatch(this::matches);
+ }
}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/SourceLocation.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/SourceLocation.java
index 3accc38..2a8636a 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/SourceLocation.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/SourceLocation.java
@@ -63,12 +63,20 @@
return column;
}
- public void writeFields(DataOutput output) throws IOException {
- output.writeInt(line);
- output.writeInt(column);
+ public static void writeFields(SourceLocation sourceLocation, DataOutput output) throws IOException {
+ if (sourceLocation == null) {
+ output.writeInt(-1);
+ } else {
+ output.writeInt(sourceLocation.getLine());
+ output.writeInt(sourceLocation.getColumn());
+ }
}
public static SourceLocation create(DataInput dataInput) throws IOException {
- return new SourceLocation(dataInput.readInt(), dataInput.readInt());
+ int row = dataInput.readInt();
+ if (row == -1) {
+ return null;
+ }
+ return new SourceLocation(row, dataInput.readInt());
}
-}
\ No newline at end of file
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/Warning.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/Warning.java
index 78fe3d8..b38c7ac 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/Warning.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/Warning.java
@@ -33,22 +33,35 @@
private final SourceLocation srcLocation;
private final int code;
private final String message;
+ private final Serializable[] params;
- private Warning(String component, SourceLocation srcLocation, int code, String message) {
+ private Warning(String component, SourceLocation srcLocation, int code, String message, Serializable... params) {
this.component = component;
this.srcLocation = srcLocation;
this.code = code;
this.message = message;
+ this.params = params;
}
+ /**
+ * @deprecated use {@link Warning#of(SourceLocation, IError, Serializable...)}
+ */
+ @Deprecated
public static Warning of(String component, SourceLocation srcLocation, int code, String message) {
- Objects.requireNonNull(srcLocation, "warnings must have source location");
return new Warning(component, srcLocation, code, message);
}
- public static Warning forHyracks(SourceLocation srcLocation, int code, Serializable... params) {
- return Warning.of(ErrorCode.HYRACKS, srcLocation, code, ErrorMessageUtil.formatMessage(ErrorCode.HYRACKS, code,
- ErrorCode.getErrorMessage(code), srcLocation, params));
+ /**
+ * @deprecated call {@link Warning#of(SourceLocation, IError, Serializable...)} directly
+ */
+ @Deprecated
+ public static Warning forHyracks(SourceLocation srcLocation, ErrorCode code, Serializable... params) {
+ return of(srcLocation, code, params);
+ }
+
+ public static Warning of(SourceLocation srcLocation, IError code, Serializable... params) {
+ return new Warning(code.component(), srcLocation, code.intValue(), ErrorMessageUtil
+ .formatMessage(code.component(), code.intValue(), code.errorMessage(), srcLocation, params), params);
}
public String getComponent() {
@@ -88,14 +101,15 @@
output.writeUTF(component);
output.writeInt(code);
output.writeUTF(message);
- srcLocation.writeFields(output);
+ SourceLocation.writeFields(srcLocation, output);
}
public static Warning create(DataInput input) throws IOException {
String comp = input.readUTF();
int code = input.readInt();
String msg = input.readUTF();
- return new Warning(comp, SourceLocation.create(input), code, msg);
+ SourceLocation sourceLocation = SourceLocation.create(input);
+ return new Warning(comp, sourceLocation, code, msg);
}
@Override
@@ -103,4 +117,8 @@
return "Warning{" + "component='" + component + '\'' + ", srcLocation=" + srcLocation + ", code=" + code
+ ", message='" + message + '\'' + '}';
}
+
+ public Serializable[] getParams() {
+ return params;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/messages/IMessage.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/messages/IMessage.java
index 94e08d7..9ca7f88 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/messages/IMessage.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/messages/IMessage.java
@@ -34,8 +34,6 @@
static void logMessage(Logger logger, IMessage msg) {
final Level logLevel = msg.isWhispered() ? Level.TRACE : Level.INFO;
- if (logger.isEnabled(logLevel)) {
- logger.info("Received message: {}", msg);
- }
+ logger.log(logLevel, "Received message: {}", msg);
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/network/INetworkSecurityConfig.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/network/INetworkSecurityConfig.java
index 95c3efc..b483158 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/network/INetworkSecurityConfig.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/network/INetworkSecurityConfig.java
@@ -19,10 +19,10 @@
package org.apache.hyracks.api.network;
import java.io.File;
+import java.io.Serializable;
import java.security.KeyStore;
-import java.util.Optional;
-public interface INetworkSecurityConfig {
+public interface INetworkSecurityConfig extends Serializable {
/**
* Indicates if SSL is enabled
@@ -65,11 +65,4 @@
* @return the trust store file
*/
File getTrustStoreFile();
-
- /**
- * Gets the password for the trust store file, if configured
- *
- * @return the password to the trust store file, if configured
- */
- Optional<String> getTrustStorePassword();
-}
\ No newline at end of file
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/CleanupUtils.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/CleanupUtils.java
index 6e6d342..220311e 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/CleanupUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/CleanupUtils.java
@@ -96,7 +96,9 @@
} catch (Throwable loggingFailure) { // NOSONAR: Ignore catching Throwable
// NOSONAR ignore logging failure
}
- root.addSuppressed(th);
+ if (root != null) {
+ root.addSuppressed(th);
+ }
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ErrorMessageUtil.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ErrorMessageUtil.java
index 8758ef7..a7372ae 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ErrorMessageUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ErrorMessageUtil.java
@@ -22,11 +22,13 @@
import java.io.IOException;
import java.io.InputStream;
import java.io.Serializable;
+import java.util.Arrays;
import java.util.Formatter;
import java.util.HashMap;
import java.util.Map;
import java.util.Properties;
+import org.apache.hyracks.api.exceptions.IError;
import org.apache.hyracks.api.exceptions.IFormattedException;
import org.apache.hyracks.api.exceptions.SourceLocation;
import org.apache.logging.log4j.Level;
@@ -110,8 +112,10 @@
return fmt.out().toString();
} catch (Exception e) {
// Do not throw further exceptions during exception processing.
- LOGGER.log(Level.WARN, e.getLocalizedMessage(), e);
- return e.getMessage();
+ final String paramsString = Arrays.toString(params);
+ LOGGER.log(Level.WARN, "error formatting {}{}: message {} params {}", component, errorCode, message,
+ paramsString, e);
+ return message + "; " + paramsString;
}
}
@@ -121,4 +125,19 @@
}
return String.valueOf(t);
}
+
+ public static String[] defineMessageEnumOrdinalMap(IError[] values, String resourcePath) {
+ String[] enumMessages = new String[values.length];
+ try (InputStream resourceStream = values[0].getClass().getClassLoader().getResourceAsStream(resourcePath)) {
+ Map<Integer, String> errorMessageMap = loadErrorMap(resourceStream);
+ for (IError error : values) {
+ enumMessages[((Enum) error).ordinal()] = errorMessageMap.computeIfAbsent(error.intValue(), intValue -> {
+ throw new IllegalStateException("error message missing for " + error + " (" + intValue + ")");
+ });
+ }
+ } catch (Exception e) {
+ throw new IllegalStateException(e);
+ }
+ return enumMessages;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java
index 21f393c..d748ed8 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ExceptionUtils.java
@@ -21,9 +21,14 @@
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
import org.apache.hyracks.api.exceptions.ErrorCode;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.util.ThrowingFunction;
+
+import com.google.common.util.concurrent.UncheckedExecutionException;
/**
* @author yingyib
@@ -65,7 +70,7 @@
newExceptions.add(e);
}
} else {
- newExceptions.add(new HyracksDataException(ErrorCode.HYRACKS, ErrorCode.FAILURE_ON_NODE, e, nodeId));
+ newExceptions.add(HyracksDataException.create(ErrorCode.FAILURE_ON_NODE, e, nodeId));
}
}
exceptions.clear();
@@ -133,4 +138,24 @@
public static boolean causedByInterrupt(Throwable th) {
return getRootCause(th) instanceof InterruptedException;
}
+
+ /**
+ * Convenience utility method to provide a form of {@link Map#computeIfAbsent(Object, Function)} which allows
+ * throwable mapping functions. Any exceptions thrown by the mapping function is propagated as an instance of
+ * {@link HyracksDataException}
+ */
+ public static <K, V> V computeIfAbsent(Map<K, V> map, K key, ThrowingFunction<K, V> function)
+ throws HyracksDataException {
+ try {
+ return map.computeIfAbsent(key, k -> {
+ try {
+ return function.process(k);
+ } catch (Exception e) {
+ throw new UncheckedExecutionException(e);
+ }
+ });
+ } catch (UncheckedExecutionException e) {
+ throw HyracksDataException.create(e.getCause());
+ }
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksThrowingConsumer.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksThrowingConsumer.java
new file mode 100644
index 0000000..b677132
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/HyracksThrowingConsumer.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.api.util;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+@FunctionalInterface
+public interface HyracksThrowingConsumer<V> {
+ void process(V value) throws HyracksDataException;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
index cfb6c9e..5774f64 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
@@ -20,6 +20,7 @@
# 0 --- 9999: runtime errors
1 = Unsupported operation %1$s in %2$s operator
2 = Error in processing tuple %1$s in a frame
+3 = Failure on node %1$s
4 = The file with absolute path %1$s is not within any of the current IO devices
5 = Phrase search in Full-text is not supported. An expression should include only one word
6 = Job queue is full with %1$s jobs
@@ -59,7 +60,7 @@
40 = Unknown inverted index type %1$s
41 = Cannot propose linearizer if dimensions have different types
42 = Cannot propose linearizer for type %1$s
-43 = Record size (%1$s) larger than maximum acceptable record size (%2$s)
+43 = Index entry size (%1$s) larger than maximum acceptable entry size (%2$s)
44 = Failed to re-find parent of a page in the tree
45 = Failed to find a tuple in a page
46 = Unsorted load input
@@ -138,8 +139,9 @@
119 = Unsupported window specification: PARTITION BY %1$s, ORDER BY %2$s
120 = End of file
121 = A numeric type promotion error has occurred: %1$s
-122 = Encountered an error while printing the plan
+122 = Encountered an error while printing the plan: %1$s
123 = Insufficient memory is provided for the join operators, please increase the join memory budget.
+124 = Parsing error at %1$s line %2$s field %3$s: %4$s
10000 = The given rule collection %1$s is not an instance of the List class.
10001 = Cannot compose partition constraint %1$s with %2$s
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/job/resource/ClusterCapacityTest.java b/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/job/resource/ClusterCapacityTest.java
index 277e8e2..e8bc546 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/job/resource/ClusterCapacityTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/job/resource/ClusterCapacityTest.java
@@ -47,14 +47,14 @@
try {
capacity.getMemoryByteSize(nodeId);
} catch (HyracksException e) {
- nodeNotExist = e.getErrorCode() == ErrorCode.NO_SUCH_NODE;
+ nodeNotExist = e.matches(ErrorCode.NO_SUCH_NODE);
}
Assert.assertTrue(nodeNotExist);
nodeNotExist = false;
try {
capacity.getCores(nodeId);
} catch (HyracksException e) {
- nodeNotExist = e.getErrorCode() == ErrorCode.NO_SUCH_NODE;
+ nodeNotExist = e.matches(ErrorCode.NO_SUCH_NODE);
}
Assert.assertTrue(nodeNotExist);
@@ -70,14 +70,14 @@
try {
capacity.getMemoryByteSize(nodeId);
} catch (HyracksException e) {
- nodeNotExist = e.getErrorCode() == ErrorCode.NO_SUCH_NODE;
+ nodeNotExist = e.matches(ErrorCode.NO_SUCH_NODE);
}
Assert.assertTrue(nodeNotExist);
nodeNotExist = false;
try {
capacity.getCores(nodeId);
} catch (HyracksException e) {
- nodeNotExist = e.getErrorCode() == ErrorCode.NO_SUCH_NODE;
+ nodeNotExist = e.matches(ErrorCode.NO_SUCH_NODE);
}
Assert.assertTrue(nodeNotExist);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/test/HyracksDataExceptionTest.java b/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/test/HyracksDataExceptionTest.java
index c8c1790..e4a49f0 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/test/HyracksDataExceptionTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/test/java/org/apache/hyracks/api/test/HyracksDataExceptionTest.java
@@ -20,7 +20,6 @@
import org.apache.hyracks.api.exceptions.ErrorCode;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.util.ErrorMessageUtil;
import org.junit.Assert;
import org.junit.Test;
@@ -32,12 +31,4 @@
HyracksDataException causeWithNodeId = HyracksDataException.create(cause, "nc1");
Assert.assertEquals(cause.getMessage(), causeWithNodeId.getMessage());
}
-
- @Test
- public void returnedMessageWithNoComponentTest() {
- HyracksDataException cause = new HyracksDataException(ErrorMessageUtil.NONE, ErrorCode.ERROR_PROCESSING_TUPLE,
- ErrorCode.getErrorMessage(ErrorCode.ERROR_PROCESSING_TUPLE), 2);
- HyracksDataException causeWithNodeId = HyracksDataException.create(cause, "nc1");
- Assert.assertEquals(cause.getMessage(), causeWithNodeId.getMessage());
- }
}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSetReader.java b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSetReader.java
index c8f5bd9..b29e2ea2 100644
--- a/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSetReader.java
+++ b/hyracks-fullstack/hyracks/hyracks-client/src/main/java/org/apache/hyracks/client/result/ResultSetReader.java
@@ -77,7 +77,7 @@
try {
return resultDirectory.getResultStatus(jobId, resultSetId);
} catch (HyracksDataException e) {
- if (e.getErrorCode() != ErrorCode.NO_RESULT_SET) {
+ if (!e.matches(ErrorCode.NO_RESULT_SET)) {
LOGGER.log(Level.WARN, "Exception retrieving result set for job " + jobId, e);
}
} catch (Exception e) {
@@ -90,36 +90,43 @@
public int read(IFrame frame) throws HyracksDataException {
frame.reset();
int readSize = 0;
- if (isFirstRead() && !hasNextRecord()) {
- return readSize;
- }
- // read until frame is full or all result records have been read
- while (readSize < frame.getFrameSize()) {
- if (currentRecordMonitor.hasMoreFrames()) {
- final ByteBuffer readBuffer = currentRecordChannel.getNextBuffer();
- if (readBuffer == null) {
- throw new IllegalStateException("Unexpected empty frame");
- }
- currentRecordMonitor.notifyFrameRead();
- if (readSize == 0) {
- final int nBlocks = FrameHelper.deserializeNumOfMinFrame(readBuffer);
- frame.ensureFrameSize(frame.getMinSize() * nBlocks);
- frame.getBuffer().clear();
- }
- frame.getBuffer().put(readBuffer);
- currentRecordChannel.recycleBuffer(readBuffer);
- readSize = frame.getBuffer().position();
- } else {
- currentRecordChannel.close();
- if (currentRecordMonitor.failed()) {
- throw HyracksDataException.create(ErrorCode.FAILED_TO_READ_RESULT, jobId);
- }
- if (isLastRecord() || !hasNextRecord()) {
- break;
+ try {
+ if (isFirstRead() && !hasNextRecord()) {
+ return readSize;
+ }
+ // read until frame is full or all result records have been read
+ while (readSize < frame.getFrameSize()) {
+ if (currentRecordMonitor.hasMoreFrames()) {
+ final ByteBuffer readBuffer = currentRecordChannel.getNextBuffer();
+ if (readBuffer == null) {
+ throw new IllegalStateException("Unexpected empty frame");
+ }
+ currentRecordMonitor.notifyFrameRead();
+ if (readSize == 0) {
+ final int nBlocks = FrameHelper.deserializeNumOfMinFrame(readBuffer);
+ frame.ensureFrameSize(frame.getMinSize() * nBlocks);
+ frame.getBuffer().clear();
+ }
+ frame.getBuffer().put(readBuffer);
+ currentRecordChannel.recycleBuffer(readBuffer);
+ readSize = frame.getBuffer().position();
+ } else {
+ currentRecordChannel.close();
+ if (currentRecordMonitor.failed()) {
+ throw HyracksDataException.create(ErrorCode.FAILED_TO_READ_RESULT, jobId);
+ }
+ if (isLastRecord() || !hasNextRecord()) {
+ break;
+ }
}
}
+ frame.getBuffer().flip();
+ } catch (Exception e) {
+ if (isLocalFailure()) {
+ currentRecordChannel.fail();
+ }
+ throw e;
}
- frame.getBuffer().flip();
return readSize;
}
@@ -128,7 +135,7 @@
try {
return resultDirectory.getResultMetadata(jobId, resultSetId);
} catch (HyracksDataException e) {
- if (e.getErrorCode() != ErrorCode.NO_RESULT_SET) {
+ if (!e.matches(ErrorCode.NO_RESULT_SET)) {
LOGGER.log(Level.WARN, "Exception retrieving result set for job " + jobId, e);
}
} catch (Exception e) {
@@ -201,6 +208,10 @@
return knownRecords != null && currentRecord == knownRecords.length - 1;
}
+ private boolean isLocalFailure() {
+ return currentRecordMonitor != null && !currentRecordMonitor.failed();
+ }
+
private static class ResultInputChannelMonitor implements IInputChannelMonitor {
private int availableFrames;
diff --git a/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/NetworkInputChannel.java b/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/NetworkInputChannel.java
index 58664c6..53bb7cd 100644
--- a/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/NetworkInputChannel.java
+++ b/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/NetworkInputChannel.java
@@ -119,6 +119,11 @@
}
+ @Override
+ public void fail() {
+ // do nothing (covered by job lifecycle)
+ }
+
private class ReadFullBufferAcceptor implements ICloseableBufferAcceptor {
@Override
public void accept(ByteBuffer buffer) {
diff --git a/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/ResultNetworkInputChannel.java b/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/ResultNetworkInputChannel.java
index 1df39e9..38cf7c4 100644
--- a/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/ResultNetworkInputChannel.java
+++ b/hyracks-fullstack/hyracks/hyracks-comm/src/main/java/org/apache/hyracks/comm/channels/ResultNetworkInputChannel.java
@@ -32,6 +32,7 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.job.JobId;
import org.apache.hyracks.api.result.ResultSetId;
+import org.apache.hyracks.net.protocols.muxdemux.AbstractChannelWriteInterface;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -127,6 +128,11 @@
}
+ @Override
+ public void fail() {
+ ccb.getWriteInterface().getFullBufferAcceptor().error(AbstractChannelWriteInterface.REMOTE_ERROR_CODE);
+ }
+
private class ReadFullBufferAcceptor implements ICloseableBufferAcceptor {
@Override
public void accept(ByteBuffer buffer) {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml
index ab67561..7ff1c4c 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/pom.xml
@@ -92,7 +92,7 @@
</dependency>
<dependency>
<groupId>org.mockito</groupId>
- <artifactId>mockito-all</artifactId>
+ <artifactId>mockito-core</artifactId>
<scope>test</scope>
</dependency>
<dependency>
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/cluster/NodeManagerTest.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/cluster/NodeManagerTest.java
index d92727c..65459c6 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/cluster/NodeManagerTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/cluster/NodeManagerTest.java
@@ -122,7 +122,7 @@
try {
nodeManager.addNode(NODE1, ncState1);
} catch (HyracksException e) {
- invalidNetworkAddress = e.getErrorCode() == ErrorCode.INVALID_NETWORK_ADDRESS;
+ invalidNetworkAddress = e.matches(ErrorCode.INVALID_NETWORK_ADDRESS);
}
Assert.assertTrue(invalidNetworkAddress);
@@ -150,7 +150,7 @@
try {
nodeManager.addNode(null, null);
} catch (HyracksException e) {
- invalidParameter = e.getErrorCode() == ErrorCode.INVALID_INPUT_PARAMETER;
+ invalidParameter = e.matches(ErrorCode.INVALID_INPUT_PARAMETER);
}
Assert.assertTrue(invalidParameter);
@@ -205,14 +205,14 @@
try {
capacity.getMemoryByteSize(nodeId);
} catch (HyracksException e) {
- nodeNotExist = e.getErrorCode() == ErrorCode.NO_SUCH_NODE;
+ nodeNotExist = e.matches(ErrorCode.NO_SUCH_NODE);
}
Assert.assertTrue(nodeNotExist);
nodeNotExist = false;
try {
capacity.getCores(nodeId);
} catch (HyracksException e) {
- nodeNotExist = e.getErrorCode() == ErrorCode.NO_SUCH_NODE;
+ nodeNotExist = e.matches(ErrorCode.NO_SUCH_NODE);
}
Assert.assertTrue(nodeNotExist);
}
@@ -230,7 +230,7 @@
try {
nodeManager.addNode(NODE2, ncState2);
} catch (HyracksException e) {
- nodeNotExist = e.getErrorCode() == ErrorCode.NO_SUCH_NODE;
+ nodeNotExist = e.matches(ErrorCode.NO_SUCH_NODE);
}
Assert.assertTrue(nodeNotExist);
Assert.assertTrue(nodeManager.getIpAddressNodeNameMap().size() == 1);
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/job/JobManagerTest.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/job/JobManagerTest.java
index 251aed8..19340d0 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/job/JobManagerTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/test/java/org/apache/hyracks/control/cc/job/JobManagerTest.java
@@ -114,7 +114,7 @@
jobManager.add(run);
} catch (HyracksException e) {
// Verifies the error code.
- jobQueueFull = e.getErrorCode() == ErrorCode.JOB_QUEUE_FULL;
+ jobQueueFull = e.matches(ErrorCode.JOB_QUEUE_FULL);
}
Assert.assertTrue(jobQueueFull);
@@ -154,7 +154,7 @@
jobManager.add(run);
} catch (HyracksException e) {
// Verifies the error code.
- rejected = e.getErrorCode() == ErrorCode.JOB_REQUIREMENTS_EXCEED_CAPACITY;
+ rejected = e.matches(ErrorCode.JOB_REQUIREMENTS_EXCEED_CAPACITY);
}
Assert.assertTrue(rejected);
Assert.assertTrue(jobManager.getRunningJobs().isEmpty());
@@ -200,7 +200,7 @@
try {
jobManager.add(null);
} catch (HyracksException e) {
- invalidParameter = e.getErrorCode() == ErrorCode.INVALID_INPUT_PARAMETER;
+ invalidParameter = e.matches(ErrorCode.INVALID_INPUT_PARAMETER);
}
Assert.assertTrue(invalidParameter);
Assert.assertTrue(jobManager.getRunningJobs().isEmpty());
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/OptionTypes.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/OptionTypes.java
index 7088e08..332d65b 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/OptionTypes.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/config/OptionTypes.java
@@ -33,66 +33,11 @@
public class OptionTypes {
- public static final IOptionType<Integer> INTEGER_BYTE_UNIT = new IOptionType<Integer>() {
- @Override
- public Integer parse(String s) {
- if (s == null) {
- return null;
- }
- long result1 = StorageUtil.getByteValue(s);
- if (result1 > Integer.MAX_VALUE || result1 < Integer.MIN_VALUE) {
- throw new IllegalArgumentException("The given value: " + result1 + " is not within the int range.");
- }
- return (int) result1;
- }
+ public static final IOptionType<Integer> INTEGER_BYTE_UNIT = new IntegerByteUnit();
+ public static final IOptionType<Integer> POSITIVE_INTEGER_BYTE_UNIT = new IntegerByteUnit(1, Integer.MAX_VALUE);
- @Override
- public Integer parse(JsonNode node) {
- return node.isNull() ? null : parse(node.asText());
- }
-
- @Override
- public Class<Integer> targetType() {
- return Integer.class;
- }
-
- @Override
- public String serializeToHumanReadable(Object value) {
- return value + " (" + StorageUtil.toHumanReadableSize((int) value) + ")";
- }
-
- @Override
- public void serializeJSONField(String fieldName, Object value, ObjectNode node) {
- node.put(fieldName, (int) value);
- }
- };
-
- public static final IOptionType<Long> LONG_BYTE_UNIT = new IOptionType<Long>() {
- @Override
- public Long parse(String s) {
- return s == null ? null : StorageUtil.getByteValue(s);
- }
-
- @Override
- public Long parse(JsonNode node) {
- return node.isNull() ? null : parse(node.asText());
- }
-
- @Override
- public Class<Long> targetType() {
- return Long.class;
- }
-
- @Override
- public String serializeToHumanReadable(Object value) {
- return value + " (" + StorageUtil.toHumanReadableSize((long) value) + ")";
- }
-
- @Override
- public void serializeJSONField(String fieldName, Object value, ObjectNode node) {
- node.put(fieldName, (long) value);
- }
- };
+ public static final IOptionType<Long> LONG_BYTE_UNIT = new LongByteUnit();
+ public static final IOptionType<Long> POSITIVE_LONG_BYTE_UNIT = new LongByteUnit(1, Long.MAX_VALUE);
public static final IOptionType<Short> SHORT = new IOptionType<Short>() {
@Override
@@ -124,27 +69,7 @@
}
};
- public static final IOptionType<Integer> INTEGER = new IOptionType<Integer>() {
- @Override
- public Integer parse(String s) {
- return Integer.parseInt(s);
- }
-
- @Override
- public Integer parse(JsonNode node) {
- return node.isNull() ? null : node.asInt();
- }
-
- @Override
- public Class<Integer> targetType() {
- return Integer.class;
- }
-
- @Override
- public void serializeJSONField(String fieldName, Object value, ObjectNode node) {
- node.put(fieldName, (int) value);
- }
- };
+ public static final IOptionType<Integer> INTEGER = new IntegerOptionType();
public static final IOptionType<Double> DOUBLE = new IOptionType<Double>() {
@Override
@@ -190,27 +115,7 @@
}
};
- public static final IOptionType<Long> LONG = new IOptionType<Long>() {
- @Override
- public Long parse(String s) {
- return Long.parseLong(s);
- }
-
- @Override
- public Long parse(JsonNode node) {
- return node.isNull() ? null : node.asLong();
- }
-
- @Override
- public Class<Long> targetType() {
- return Long.class;
- }
-
- @Override
- public void serializeJSONField(String fieldName, Object value, ObjectNode node) {
- node.put(fieldName, (long) value);
- }
- };
+ public static final IOptionType<Long> LONG = new LongOptionType();
public static final IOptionType<Boolean> BOOLEAN = new IOptionType<Boolean>() {
@Override
@@ -287,7 +192,7 @@
List<String> strings = new ArrayList<>();
if (node instanceof ArrayNode) {
node.elements().forEachRemaining(n -> strings.add(n.asText()));
- return strings.toArray(new String[strings.size()]);
+ return strings.toArray(new String[0]);
} else {
return parse(node.asText());
}
@@ -340,54 +245,186 @@
}
};
- public static final IOptionType<Integer> UNSIGNED_INTEGER = new IOptionType<Integer>() {
- @Override
- public Integer parse(String s) {
- return Integer.parseUnsignedInt(s);
- }
+ public static final IOptionType<Integer> NONNEGATIVE_INTEGER = getRangedIntegerType(0, Integer.MAX_VALUE);
- @Override
- public Integer parse(JsonNode node) {
- return node.isNull() ? null : parse(node.asText());
- }
-
- @Override
- public Class<Integer> targetType() {
- return Integer.class;
- }
-
- @Override
- public void serializeJSONField(String fieldName, Object value, ObjectNode node) {
- node.put(fieldName, (int) value);
- }
- };
-
- public static final IOptionType<Integer> POSITIVE_INTEGER = new IOptionType<Integer>() {
- @Override
- public Integer parse(String s) {
- final int value = Integer.parseUnsignedInt(s);
- if (value == 0) {
- throw new IllegalArgumentException("Value must be greater than zero");
- }
- return value;
- }
-
- @Override
- public Integer parse(JsonNode node) {
- return node.isNull() ? null : parse(node.asText());
- }
-
- @Override
- public Class<Integer> targetType() {
- return Integer.class;
- }
-
- @Override
- public void serializeJSONField(String fieldName, Object value, ObjectNode node) {
- node.put(fieldName, (int) value);
- }
- };
+ public static final IOptionType<Integer> POSITIVE_INTEGER = getRangedIntegerType(1, Integer.MAX_VALUE);
private OptionTypes() {
}
+
+ public static IOptionType<Integer> getRangedIntegerType(final int minValueInclusive, final int maxValueInclusive) {
+ return new RangedIntegerOptionType(minValueInclusive, maxValueInclusive);
+ }
+
+ public static class IntegerOptionType implements IOptionType<Integer> {
+ @Override
+ public Integer parse(String s) {
+ return Integer.parseInt(s);
+ }
+
+ @Override
+ public Integer parse(JsonNode node) {
+ return node.isNull() ? null : node.asInt();
+ }
+
+ @Override
+ public Class<Integer> targetType() {
+ return Integer.class;
+ }
+
+ @Override
+ public void serializeJSONField(String fieldName, Object value, ObjectNode node) {
+ node.put(fieldName, (int) value);
+ }
+ }
+
+ private static class RangedIntegerOptionType extends IntegerOptionType {
+ private final int minValue;
+ private final int maxValue;
+
+ RangedIntegerOptionType(int minValue, int maxValue) {
+ this.minValue = minValue;
+ this.maxValue = maxValue;
+ }
+
+ @Override
+ public Integer parse(String value) {
+ int intValue = super.parse(value);
+ rangeCheck(intValue);
+ return intValue;
+ }
+
+ void rangeCheck(long intValue) {
+ if (intValue < minValue || intValue > maxValue) {
+ if (maxValue == Integer.MAX_VALUE) {
+ if (minValue == 0) {
+ throw new IllegalArgumentException("integer value must not be negative, but was " + intValue);
+ } else if (minValue == 1) {
+ throw new IllegalArgumentException(
+ "integer value must be greater than zero, but was " + intValue);
+ }
+ }
+ throw new IllegalArgumentException(
+ "integer value must be between " + minValue + "-" + maxValue + " (inclusive)");
+ }
+ }
+ }
+
+ private static class IntegerByteUnit extends RangedIntegerOptionType {
+
+ IntegerByteUnit() {
+ this(Integer.MIN_VALUE, Integer.MAX_VALUE);
+ }
+
+ IntegerByteUnit(int minValue, int maxValue) {
+ super(minValue, maxValue);
+ }
+
+ @Override
+ public Integer parse(String s) {
+ if (s == null) {
+ return null;
+ }
+ long result = StorageUtil.getByteValue(s);
+ rangeCheck(result);
+ return (int) result;
+ }
+
+ @Override
+ public Integer parse(JsonNode node) {
+ // TODO: we accept human readable sizes from json- why not emit human readable sizes?
+ return node.isNull() ? null : parse(node.asText());
+ }
+
+ @Override
+ public String serializeToHumanReadable(Object value) {
+ return value + " (" + StorageUtil.toHumanReadableSize((int) value) + ")";
+ }
+ }
+
+ private static class RangedLongOptionType extends LongOptionType {
+ private final long minValue;
+ private final long maxValue;
+
+ RangedLongOptionType(long minValue, long maxValue) {
+ this.minValue = minValue;
+ this.maxValue = maxValue;
+ }
+
+ @Override
+ public Long parse(String value) {
+ long longValue = super.parse(value);
+ rangeCheck(longValue);
+ return longValue;
+ }
+
+ void rangeCheck(long longValue) {
+ if (longValue < minValue || longValue > maxValue) {
+ if (maxValue == Long.MAX_VALUE) {
+ if (minValue == 0) {
+ throw new IllegalArgumentException("long value must not be negative, but was " + longValue);
+ } else if (minValue == 1) {
+ throw new IllegalArgumentException(
+ "long value must be greater than zero, but was " + longValue);
+ }
+ }
+ throw new IllegalArgumentException(
+ "long value must be between " + minValue + "-" + maxValue + " (inclusive)");
+ }
+ }
+ }
+
+ private static class LongByteUnit extends RangedLongOptionType {
+
+ LongByteUnit() {
+ this(Long.MIN_VALUE, Long.MAX_VALUE);
+ }
+
+ LongByteUnit(long minValue, long maxValue) {
+ super(minValue, maxValue);
+ }
+
+ @Override
+ public Long parse(String s) {
+ if (s == null) {
+ return null;
+ }
+ long result = StorageUtil.getByteValue(s);
+ rangeCheck(result);
+ return result;
+ }
+
+ @Override
+ public Long parse(JsonNode node) {
+ // TODO: we accept human readable sizes from json- why not emit human readable sizes?
+ return node.isNull() ? null : parse(node.asText());
+ }
+
+ @Override
+ public String serializeToHumanReadable(Object value) {
+ return value + " (" + StorageUtil.toHumanReadableSize((long) value) + ")";
+ }
+ }
+
+ private static class LongOptionType implements IOptionType<Long> {
+ @Override
+ public Long parse(String s) {
+ return Long.parseLong(s);
+ }
+
+ @Override
+ public Long parse(JsonNode node) {
+ return node.isNull() ? null : node.asLong();
+ }
+
+ @Override
+ public Class<Long> targetType() {
+ return Long.class;
+ }
+
+ @Override
+ public void serializeJSONField(String fieldName, Object value, ObjectNode node) {
+ node.put(fieldName, (long) value);
+ }
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/CCConfig.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/CCConfig.java
index 0de75d9..17be02e 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/CCConfig.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/CCConfig.java
@@ -20,10 +20,10 @@
import static org.apache.hyracks.control.common.config.OptionTypes.BOOLEAN;
import static org.apache.hyracks.control.common.config.OptionTypes.LONG;
+import static org.apache.hyracks.control.common.config.OptionTypes.NONNEGATIVE_INTEGER;
import static org.apache.hyracks.control.common.config.OptionTypes.POSITIVE_INTEGER;
import static org.apache.hyracks.control.common.config.OptionTypes.SHORT;
import static org.apache.hyracks.control.common.config.OptionTypes.STRING;
-import static org.apache.hyracks.control.common.config.OptionTypes.UNSIGNED_INTEGER;
import java.io.File;
import java.net.InetAddress;
@@ -49,22 +49,22 @@
ADDRESS(STRING, InetAddress.getLoopbackAddress().getHostAddress()),
PUBLIC_ADDRESS(STRING, ADDRESS),
CLUSTER_LISTEN_ADDRESS(STRING, ADDRESS),
- CLUSTER_LISTEN_PORT(UNSIGNED_INTEGER, 1099),
+ CLUSTER_LISTEN_PORT(NONNEGATIVE_INTEGER, 1099),
CLUSTER_PUBLIC_ADDRESS(STRING, PUBLIC_ADDRESS),
- CLUSTER_PUBLIC_PORT(UNSIGNED_INTEGER, CLUSTER_LISTEN_PORT),
+ CLUSTER_PUBLIC_PORT(NONNEGATIVE_INTEGER, CLUSTER_LISTEN_PORT),
CLIENT_LISTEN_ADDRESS(STRING, ADDRESS),
- CLIENT_LISTEN_PORT(UNSIGNED_INTEGER, 1098),
+ CLIENT_LISTEN_PORT(NONNEGATIVE_INTEGER, 1098),
CLIENT_PUBLIC_ADDRESS(STRING, PUBLIC_ADDRESS),
- CLIENT_PUBLIC_PORT(UNSIGNED_INTEGER, CLIENT_LISTEN_PORT),
+ CLIENT_PUBLIC_PORT(NONNEGATIVE_INTEGER, CLIENT_LISTEN_PORT),
CONSOLE_LISTEN_ADDRESS(STRING, ADDRESS),
- CONSOLE_LISTEN_PORT(UNSIGNED_INTEGER, 16001),
+ CONSOLE_LISTEN_PORT(NONNEGATIVE_INTEGER, 16001),
CONSOLE_PUBLIC_ADDRESS(STRING, PUBLIC_ADDRESS),
- CONSOLE_PUBLIC_PORT(UNSIGNED_INTEGER, CONSOLE_LISTEN_PORT),
+ CONSOLE_PUBLIC_PORT(NONNEGATIVE_INTEGER, CONSOLE_LISTEN_PORT),
HEARTBEAT_PERIOD(LONG, 10000L), // TODO (mblow): add time unit
- HEARTBEAT_MAX_MISSES(UNSIGNED_INTEGER, 5),
+ HEARTBEAT_MAX_MISSES(NONNEGATIVE_INTEGER, 5),
DEAD_NODE_SWEEP_THRESHOLD(LONG, HEARTBEAT_PERIOD),
- PROFILE_DUMP_PERIOD(UNSIGNED_INTEGER, 0),
- JOB_HISTORY_SIZE(UNSIGNED_INTEGER, 10),
+ PROFILE_DUMP_PERIOD(NONNEGATIVE_INTEGER, 0),
+ JOB_HISTORY_SIZE(NONNEGATIVE_INTEGER, 10),
RESULT_TTL(LONG, 86400000L), // TODO(mblow): add time unit
RESULT_SWEEP_THRESHOLD(LONG, 60000L), // TODO(mblow): add time unit
@SuppressWarnings("RedundantCast") // not redundant- false positive from IDEA
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
index e947d7a..ef244c59 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
@@ -21,10 +21,10 @@
import static org.apache.hyracks.control.common.config.OptionTypes.INTEGER;
import static org.apache.hyracks.control.common.config.OptionTypes.INTEGER_BYTE_UNIT;
import static org.apache.hyracks.control.common.config.OptionTypes.LONG;
+import static org.apache.hyracks.control.common.config.OptionTypes.NONNEGATIVE_INTEGER;
import static org.apache.hyracks.control.common.config.OptionTypes.POSITIVE_INTEGER;
import static org.apache.hyracks.control.common.config.OptionTypes.STRING;
import static org.apache.hyracks.control.common.config.OptionTypes.STRING_ARRAY;
-import static org.apache.hyracks.control.common.config.OptionTypes.UNSIGNED_INTEGER;
import java.net.InetAddress;
import java.util.ArrayList;
@@ -45,31 +45,31 @@
ADDRESS(STRING, InetAddress.getLoopbackAddress().getHostAddress()),
PUBLIC_ADDRESS(STRING, ADDRESS),
CLUSTER_LISTEN_ADDRESS(STRING, ADDRESS),
- CLUSTER_LISTEN_PORT(UNSIGNED_INTEGER, 0),
+ CLUSTER_LISTEN_PORT(NONNEGATIVE_INTEGER, 0),
NCSERVICE_ADDRESS(STRING, PUBLIC_ADDRESS),
NCSERVICE_PORT(INTEGER, 9090),
CLUSTER_ADDRESS(STRING, (String) null),
- CLUSTER_PORT(UNSIGNED_INTEGER, 1099),
+ CLUSTER_PORT(NONNEGATIVE_INTEGER, 1099),
CLUSTER_PUBLIC_ADDRESS(STRING, PUBLIC_ADDRESS),
- CLUSTER_PUBLIC_PORT(UNSIGNED_INTEGER, CLUSTER_LISTEN_PORT),
+ CLUSTER_PUBLIC_PORT(NONNEGATIVE_INTEGER, CLUSTER_LISTEN_PORT),
NODE_ID(STRING, (String) null),
DATA_LISTEN_ADDRESS(STRING, ADDRESS),
- DATA_LISTEN_PORT(UNSIGNED_INTEGER, 0),
+ DATA_LISTEN_PORT(NONNEGATIVE_INTEGER, 0),
DATA_PUBLIC_ADDRESS(STRING, PUBLIC_ADDRESS),
- DATA_PUBLIC_PORT(UNSIGNED_INTEGER, DATA_LISTEN_PORT),
+ DATA_PUBLIC_PORT(NONNEGATIVE_INTEGER, DATA_LISTEN_PORT),
RESULT_LISTEN_ADDRESS(STRING, ADDRESS),
- RESULT_LISTEN_PORT(UNSIGNED_INTEGER, 0),
+ RESULT_LISTEN_PORT(NONNEGATIVE_INTEGER, 0),
RESULT_PUBLIC_ADDRESS(STRING, PUBLIC_ADDRESS),
- RESULT_PUBLIC_PORT(UNSIGNED_INTEGER, RESULT_LISTEN_PORT),
+ RESULT_PUBLIC_PORT(NONNEGATIVE_INTEGER, RESULT_LISTEN_PORT),
MESSAGING_LISTEN_ADDRESS(STRING, ADDRESS),
- MESSAGING_LISTEN_PORT(UNSIGNED_INTEGER, 0),
+ MESSAGING_LISTEN_PORT(NONNEGATIVE_INTEGER, 0),
MESSAGING_PUBLIC_ADDRESS(STRING, PUBLIC_ADDRESS),
- MESSAGING_PUBLIC_PORT(UNSIGNED_INTEGER, MESSAGING_LISTEN_PORT),
+ MESSAGING_PUBLIC_PORT(NONNEGATIVE_INTEGER, MESSAGING_LISTEN_PORT),
REPLICATION_LISTEN_ADDRESS(STRING, ADDRESS),
- REPLICATION_LISTEN_PORT(UNSIGNED_INTEGER, 2000),
+ REPLICATION_LISTEN_PORT(NONNEGATIVE_INTEGER, 2000),
REPLICATION_PUBLIC_ADDRESS(STRING, PUBLIC_ADDRESS),
- REPLICATION_PUBLIC_PORT(UNSIGNED_INTEGER, REPLICATION_LISTEN_PORT),
- CLUSTER_CONNECT_RETRIES(UNSIGNED_INTEGER, 5),
+ REPLICATION_PUBLIC_PORT(NONNEGATIVE_INTEGER, REPLICATION_LISTEN_PORT),
+ CLUSTER_CONNECT_RETRIES(NONNEGATIVE_INTEGER, 5),
IODEVICES(
STRING_ARRAY,
appConfig -> new String[] {
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/heartbeat/HeartbeatManager.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/heartbeat/HeartbeatManager.java
index 08b8c11..20c1576 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/heartbeat/HeartbeatManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/heartbeat/HeartbeatManager.java
@@ -64,12 +64,12 @@
public void notifyAck(HyracksDataException exception) {
LOGGER.debug("ack rec'd from {} w/ exception: {}", ccId::toString, () -> String.valueOf(exception));
- if (exception != null && exception.matches(ErrorCode.HYRACKS, ErrorCode.NO_SUCH_NODE)) {
+ if (exception != null && exception.matches(ErrorCode.NO_SUCH_NODE)) {
LOGGER.info("{} indicates it does not recognize us; force a reconnect", ccId);
try {
ccc.forceReregister(ncs);
} catch (Exception e) {
- LOGGER.warn("ignoring exception attempting to reregister with {}", ccId, e);
+ LOGGER.warn("ignoring exception attempting to re-register with {}", ccId, e);
}
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java
index 83677f8..2a40eb8 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java
@@ -99,6 +99,11 @@
}
+ @Override
+ public void fail() {
+ // do nothing (covered by job lifecycle)
+ }
+
private class FrameWriter implements IFrameWriter {
@Override
public void open() throws HyracksDataException {
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringPointable.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringPointable.java
index 9ff2f41..eff71de 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringPointable.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/primitive/UTF8StringPointable.java
@@ -368,7 +368,9 @@
return false;
}
- builder.reset(out, Math.min(utfLen - byteIdx, (int) (charLength * 1.0 * byteIdx / chIdx)));
+ // for byteIdx = 0, this estimate assumes that every char size = 1 byte
+ int estimateOutBytes = byteIdx == 0 ? charLength : (int) (charLength * 1.0 * byteIdx / chIdx);
+ builder.reset(out, Math.min(utfLen - byteIdx, estimateOutBytes));
chIdx = 0;
while (byteIdx < utfLen && chIdx < charLength) {
builder.appendChar(src.charAt(src.getMetaDataLength() + byteIdx));
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/AbstractVarLenObjectBuilder.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/AbstractVarLenObjectBuilder.java
index 452710e..d188f16 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/AbstractVarLenObjectBuilder.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/AbstractVarLenObjectBuilder.java
@@ -85,8 +85,9 @@
for (int i = 0; i < diff; i++) {
out.writeByte(0);
}
- for (int i = ary.getLength() - 1; i >= actualDataStart + diff; i--) {
- ary.getByteArray()[i] = ary.getByteArray()[i - diff];
+ int firstCharIdx = startOffset + estimateMetaLen;
+ for (int dest = ary.getLength() - 1, src = dest - diff; src >= firstCharIdx; dest--, src--) {
+ ary.getByteArray()[dest] = ary.getByteArray()[src];
}
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/test/java/org/apache/hyracks/data/std/primitive/UTF8StringPointableTest.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/test/java/org/apache/hyracks/data/std/primitive/UTF8StringPointableTest.java
index 302e7a0..ed439cb 100644
--- a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/test/java/org/apache/hyracks/data/std/primitive/UTF8StringPointableTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/test/java/org/apache/hyracks/data/std/primitive/UTF8StringPointableTest.java
@@ -257,4 +257,37 @@
assertEquals(0, expected.compareTo(result));
}
+ @Test
+ public void testStringBuilder() throws Exception {
+ UTF8StringBuilder builder = new UTF8StringBuilder();
+ GrowableArray array = new GrowableArray();
+ UTF8StringPointable stringPointable = new UTF8StringPointable();
+ String writtenString;
+ int startIdx;
+
+ array.append(STRING_UTF8_MIX.getByteArray(), STRING_UTF8_MIX.getStartOffset(), STRING_UTF8_MIX.getLength());
+ String chunk = "ABC";
+ String originalString = chunk.repeat(699051);
+
+ // test grow path
+ startIdx = array.getLength();
+ builder.reset(array, 2);
+ builder.appendString(originalString);
+ builder.finish();
+ stringPointable.set(array.getByteArray(), startIdx, array.getLength());
+ writtenString = stringPointable.toString();
+ assertEquals(originalString, writtenString);
+
+ // test shrink path
+ array.reset();
+ array.append(STRING_UTF8_MIX.getByteArray(), STRING_UTF8_MIX.getStartOffset(), STRING_UTF8_MIX.getLength());
+ startIdx = array.getLength();
+ builder.reset(array, 699051);
+ builder.appendString(chunk);
+ builder.finish();
+ stringPointable.set(array.getByteArray(), startIdx, array.getLength());
+ writtenString = stringPointable.toString();
+ assertEquals(chunk, writtenString);
+ }
+
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/BooleanParserFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/BooleanParserFactory.java
new file mode 100644
index 0000000..141b99f
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/BooleanParserFactory.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.dataflow.common.data.parsers;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class BooleanParserFactory implements IValueParserFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ public static final IValueParserFactory INSTANCE = new BooleanParserFactory();
+
+ private BooleanParserFactory() {
+ }
+
+ @Override
+ public IValueParser createValueParser() {
+ return BooleanParserFactory::parse;
+ }
+
+ public static boolean parse(char[] buffer, int start, int length, DataOutput out) throws HyracksDataException {
+ char ch;
+ int i = start;
+ int end = start + length;
+ while (i < end && ((ch = buffer[i]) == ' ' || ch == '\t' || ch == '\n' || ch == '\r' || ch == '\f')) {
+ i++;
+ }
+ int remainingLength = end - i;
+ boolean gotBoolean = false;
+ boolean booleanValue = false;
+ if (remainingLength >= 4 && ((ch = buffer[i]) == 't' || ch == 'T') && ((ch = buffer[i + 1]) == 'r' || ch == 'R')
+ && ((ch = buffer[i + 2]) == 'u' || ch == 'U') && ((ch = buffer[i + 3]) == 'e' || ch == 'E')) {
+ gotBoolean = true;
+ booleanValue = true;
+ i = i + 4;
+ } else if (remainingLength >= 5 && ((ch = buffer[i]) == 'f' || ch == 'F')
+ && ((ch = buffer[i + 1]) == 'a' || ch == 'A') && ((ch = buffer[i + 2]) == 'l' || ch == 'L')
+ && ((ch = buffer[i + 3]) == 's' || ch == 'S') && ((ch = buffer[i + 4]) == 'e' || ch == 'E')) {
+ gotBoolean = true;
+ booleanValue = false;
+ i = i + 5;
+ }
+
+ for (; i < end; ++i) {
+ ch = buffer[i];
+ if (ch != ' ' && ch != '\t' && ch != '\n' && ch != '\r' && ch != '\f') {
+ return false;
+ }
+ }
+ if (!gotBoolean) {
+ return false;
+ }
+ try {
+ out.writeBoolean(booleanValue);
+ return true;
+ } catch (IOException e) {
+ throw HyracksDataException.create(e);
+ }
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/ByteArrayBase64ParserFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/ByteArrayBase64ParserFactory.java
index 69db7f3..cc5b68c 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/ByteArrayBase64ParserFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/ByteArrayBase64ParserFactory.java
@@ -41,11 +41,12 @@
ByteArraySerializerDeserializer serializer = ByteArraySerializerDeserializer.INSTANCE;
@Override
- public void parse(char[] input, int start, int length, DataOutput out) throws HyracksDataException {
+ public boolean parse(char[] input, int start, int length, DataOutput out) throws HyracksDataException {
parser.generatePureByteArrayFromBase64String(input, start, length);
try {
serializer.serialize(parser.getByteArray(), 0, parser.getLength(), out);
+ return true;
} catch (IOException e) {
throw HyracksDataException.create(e);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/ByteArrayHexParserFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/ByteArrayHexParserFactory.java
index c15b2ff..2191c75 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/ByteArrayHexParserFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/ByteArrayHexParserFactory.java
@@ -40,10 +40,11 @@
ByteArraySerializerDeserializer serializer = ByteArraySerializerDeserializer.INSTANCE;
@Override
- public void parse(char[] input, int start, int length, DataOutput out) throws HyracksDataException {
+ public boolean parse(char[] input, int start, int length, DataOutput out) throws HyracksDataException {
try {
parser.generateByteArrayFromHexString(input, start, length);
serializer.serialize(parser.getByteArray(), 0, parser.getLength(), out);
+ return true;
} catch (IOException e) {
throw HyracksDataException.create(e);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/DoubleParserFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/DoubleParserFactory.java
index 8998798..2004397 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/DoubleParserFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/DoubleParserFactory.java
@@ -35,12 +35,13 @@
public IValueParser createValueParser() {
return new IValueParser() {
@Override
- public void parse(char[] buffer, int start, int length, DataOutput out) throws HyracksDataException {
+ public boolean parse(char[] buffer, int start, int length, DataOutput out) throws HyracksDataException {
String s = String.valueOf(buffer, start, length);
try {
out.writeDouble(Double.parseDouble(s));
+ return true;
} catch (NumberFormatException e) {
- throw HyracksDataException.create(e);
+ return false;
} catch (IOException e) {
throw HyracksDataException.create(e);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/FloatParserFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/FloatParserFactory.java
index 414946c..2b476f1 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/FloatParserFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/FloatParserFactory.java
@@ -35,12 +35,13 @@
public IValueParser createValueParser() {
return new IValueParser() {
@Override
- public void parse(char[] buffer, int start, int length, DataOutput out) throws HyracksDataException {
+ public boolean parse(char[] buffer, int start, int length, DataOutput out) throws HyracksDataException {
String s = String.valueOf(buffer, start, length);
try {
out.writeFloat(Float.parseFloat(s));
+ return true;
} catch (NumberFormatException e) {
- throw HyracksDataException.create(e);
+ return false;
} catch (IOException e) {
throw HyracksDataException.create(e);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/IValueParser.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/IValueParser.java
index 0fcfa90..7b0090a 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/IValueParser.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/IValueParser.java
@@ -23,5 +23,5 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
public interface IValueParser {
- void parse(char[] buffer, int start, int length, DataOutput out) throws HyracksDataException;
+ boolean parse(char[] buffer, int start, int length, DataOutput out) throws HyracksDataException;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/IntegerParserFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/IntegerParserFactory.java
index c6cffb4..450aa70 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/IntegerParserFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/IntegerParserFactory.java
@@ -35,91 +35,61 @@
public IValueParser createValueParser() {
return new IValueParser() {
@Override
- public void parse(char[] buffer, int start, int length, DataOutput out) throws HyracksDataException {
- int n = 0;
- int sign = 1;
- int i = 0;
- boolean pre = true;
- for (; pre && i < length; ++i) {
- char ch = buffer[i + start];
- switch (ch) {
- case ' ':
- case '\t':
- case '\n':
- case '\r':
- case '\f':
- break;
-
- case '-':
- sign = -1;
- pre = false;
- break;
-
- case '0':
- case '1':
- case '2':
- case '3':
- case '4':
- case '5':
- case '6':
- case '7':
- case '8':
- case '9':
- pre = false;
- n = n * 10 + (ch - '0');
- break;
-
- default:
- String errorString = new String(buffer, i + start, length - i);
- throw new HyracksDataException(
- "Integer Parser - a digit is expected. But, encountered this character: " + ch
- + " in the incoming input: " + errorString);
+ public boolean parse(char[] buffer, int start, int length, DataOutput out) throws HyracksDataException {
+ // accumulating negatively like Integer.parse() to avoid surprises near MAX_VALUE
+ char c;
+ int i = start;
+ int end = start + length;
+ while (i < end && ((c = buffer[i]) == ' ' || c == '\t' || c == '\n' || c == '\r' || c == '\f')) {
+ i++;
+ }
+ boolean negative = false;
+ int limit = -Integer.MAX_VALUE;
+ if (i < end) {
+ c = buffer[i];
+ if (c == '-') {
+ negative = true;
+ limit = Integer.MIN_VALUE;
+ i++;
+ }
+ if (c == '+') {
+ i++;
}
}
- boolean post = false;
- for (; !post && i < length; ++i) {
- char ch = buffer[i + start];
- switch (ch) {
- case '0':
- case '1':
- case '2':
- case '3':
- case '4':
- case '5':
- case '6':
- case '7':
- case '8':
- case '9':
- n = n * 10 + (ch - '0');
- break;
- default:
- String errorString = new String(buffer, i + start, length - i);
- throw new HyracksDataException(
- "Integer Parser - a digit is expected. But, encountered this character: " + ch
- + " in the incoming input: " + errorString);
+ int result = 0;
+ int multiplicationMin = limit / 10;
+ boolean gotNumber = false;
+ for (; i < end; i++) {
+ c = buffer[i];
+ if (c >= '0' && c <= '9') {
+ gotNumber = true;
+ if (result < multiplicationMin) {
+ return false;
+ }
+ result *= 10;
+ int digit = c - '0';
+ if (result < limit + digit) {
+ return false;
+ }
+ result -= digit;
+ } else {
+ break;
}
}
- for (; i < length; ++i) {
- char ch = buffer[i + start];
- switch (ch) {
- case ' ':
- case '\t':
- case '\n':
- case '\r':
- case '\f':
- break;
-
- default:
- String errorString = new String(buffer, i + start, length - i);
- throw new HyracksDataException("Integer Parser - a whitespace, tab, new line, or "
- + "form-feed expected. But, encountered this character: " + ch
- + " in the incoming input: " + errorString);
+ for (; i < end; ++i) {
+ c = buffer[i];
+ if (c != ' ' && c != '\t' && c != '\n' && c != '\r' && c != '\f') {
+ return false;
}
}
+ if (!gotNumber) {
+ return false;
+ }
try {
- out.writeInt(n * sign);
+ out.writeInt(negative ? result : -result);
+ return true;
} catch (IOException e) {
throw HyracksDataException.create(e);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/LongParserFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/LongParserFactory.java
index fd47475..2cd350c 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/LongParserFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/LongParserFactory.java
@@ -35,92 +35,61 @@
public IValueParser createValueParser() {
return new IValueParser() {
@Override
- public void parse(char[] buffer, int start, int length, DataOutput out) throws HyracksDataException {
- long n = 0;
- int sign = 1;
- int i = 0;
- boolean pre = true;
- for (; pre && i < length; ++i) {
- char ch = buffer[i + start];
- switch (ch) {
- case ' ':
- case '\t':
- case '\n':
- case '\r':
- case '\f':
- break;
-
- case '-':
- sign = -1;
- pre = false;
- break;
-
- case '0':
- case '1':
- case '2':
- case '3':
- case '4':
- case '5':
- case '6':
- case '7':
- case '8':
- case '9':
- pre = false;
- n = n * 10 + (ch - '0');
- break;
-
- default:
- String errorString = new String(buffer, i + start, length - i);
- throw new HyracksDataException(
- "Long Parser - a digit is expected. But, encountered this character: " + ch
- + " in the incoming input: " + errorString);
+ public boolean parse(char[] buffer, int start, int length, DataOutput out) throws HyracksDataException {
+ // accumulating negatively like Long.parse() to avoid surprises near MAX_VALUE
+ char c;
+ int i = start;
+ int end = start + length;
+ while (i < end && ((c = buffer[i]) == ' ' || c == '\t' || c == '\n' || c == '\r' || c == '\f')) {
+ i++;
+ }
+ boolean negative = false;
+ long limit = -Long.MAX_VALUE;
+ if (i < end) {
+ c = buffer[i];
+ if (c == '-') {
+ negative = true;
+ limit = Long.MIN_VALUE;
+ i++;
+ }
+ if (c == '+') {
+ i++;
}
}
- boolean post = false;
- for (; !post && i < length; ++i) {
- char ch = buffer[i + start];
- switch (ch) {
- case '0':
- case '1':
- case '2':
- case '3':
- case '4':
- case '5':
- case '6':
- case '7':
- case '8':
- case '9':
- n = n * 10 + (ch - '0');
- break;
- default:
- String errorString = new String(buffer, i + start, length - i);
- throw new HyracksDataException(
- "Long Parser - a digit is expected. But, encountered this character: " + ch
- + " in the incoming input: " + errorString);
+ long result = 0;
+ long multiplicationMin = limit / 10;
+ boolean gotNumber = false;
+ for (; i < end; i++) {
+ c = buffer[i];
+ if (c >= '0' && c <= '9') {
+ gotNumber = true;
+ if (result < multiplicationMin) {
+ return false;
+ }
+ result *= 10;
+ int digit = c - '0';
+ if (result < limit + digit) {
+ return false;
+ }
+ result -= digit;
+ } else {
+ break;
}
}
- for (; i < length; ++i) {
- char ch = buffer[i + start];
- switch (ch) {
- case ' ':
- case '\t':
- case '\n':
- case '\r':
- case '\f':
- break;
-
- default:
- String errorString = new String(buffer, i + start, length - i);
- throw new HyracksDataException(
- "Long Parser - a whitespace, tab, new line, or form-feed expected. "
- + "But, encountered this character: " + ch + " in the incoming input: "
- + errorString);
+ for (; i < end; ++i) {
+ c = buffer[i];
+ if (c != ' ' && c != '\t' && c != '\n' && c != '\r' && c != '\f') {
+ return false;
}
}
+ if (!gotNumber) {
+ return false;
+ }
try {
- out.writeLong(n * sign);
+ out.writeLong(negative ? result : -result);
+ return true;
} catch (IOException e) {
throw HyracksDataException.create(e);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/UTF8StringParserFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/UTF8StringParserFactory.java
index 7848500..22f98a6 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/UTF8StringParserFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/parsers/UTF8StringParserFactory.java
@@ -38,9 +38,10 @@
private UTF8StringWriter writer = new UTF8StringWriter();
@Override
- public void parse(char[] buffer, int start, int length, DataOutput out) throws HyracksDataException {
+ public boolean parse(char[] buffer, int start, int length, DataOutput out) throws HyracksDataException {
try {
writer.writeUTF8(buffer, start, length, out);
+ return true;
} catch (IOException e) {
throw HyracksDataException.create(e);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileWriter.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileWriter.java
index dabdd4f..0dbd127 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/io/RunFileWriter.java
@@ -75,12 +75,21 @@
}
public void erase() throws HyracksDataException {
- close();
- file.delete();
+ try {
+ close();
+ } finally {
+ // Make sure we never access the file if it is deleted.
+ handle = null;
+ eraseClosed();
+ }
+ }
- // Make sure we never access the file if it is deleted.
- file = null;
- handle = null;
+ public void eraseClosed() {
+ FileReference f = file;
+ if (f != null) {
+ file = null;
+ f.delete();
+ }
}
public FileReference getFileReference() {
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/test/java/org/apache/hyracks/dataflow/common/data/parsers/ByteArrayBase64ParserFactoryTest.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/test/java/org/apache/hyracks/dataflow/common/data/parsers/ByteArrayBase64ParserFactoryTest.java
index 29a6d6d..5fcaf65 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/test/java/org/apache/hyracks/dataflow/common/data/parsers/ByteArrayBase64ParserFactoryTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/test/java/org/apache/hyracks/dataflow/common/data/parsers/ByteArrayBase64ParserFactoryTest.java
@@ -66,7 +66,8 @@
DataOutputStream outputStream = new DataOutputStream(bos);
ByteArrayPointable bytePtr = new ByteArrayPointable();
- parser.parse(test.toCharArray(), 0, test.length(), outputStream);
+ boolean result = parser.parse(test.toCharArray(), 0, test.length(), outputStream);
+ assertTrue(result);
bytePtr.set(bos.toByteArray(), 0, bos.size());
byte[] answer = DatatypeConverter.parseBase64Binary(test);
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/test/java/org/apache/hyracks/dataflow/common/data/parsers/ByteArrayHexParserFactoryTest.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/test/java/org/apache/hyracks/dataflow/common/data/parsers/ByteArrayHexParserFactoryTest.java
index c959c8d..a6196e7 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/test/java/org/apache/hyracks/dataflow/common/data/parsers/ByteArrayHexParserFactoryTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/test/java/org/apache/hyracks/dataflow/common/data/parsers/ByteArrayHexParserFactoryTest.java
@@ -54,7 +54,8 @@
DataOutputStream outputStream = new DataOutputStream(bos);
ByteArrayPointable bytePtr = new ByteArrayPointable();
- parser.parse(test.toCharArray(), 0, test.length(), outputStream);
+ boolean result = parser.parse(test.toCharArray(), 0, test.length(), outputStream);
+ assertTrue(result);
bytePtr.set(bos.toByteArray(), 0, bos.size());
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/test/java/org/apache/hyracks/dataflow/common/data/parsers/ParserFactoryTest.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/test/java/org/apache/hyracks/dataflow/common/data/parsers/ParserFactoryTest.java
new file mode 100644
index 0000000..e3e77da
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/test/java/org/apache/hyracks/dataflow/common/data/parsers/ParserFactoryTest.java
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.hyracks.dataflow.common.data.parsers;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.primitive.BooleanPointable;
+import org.apache.hyracks.data.std.primitive.IntegerPointable;
+import org.apache.hyracks.data.std.primitive.LongPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.junit.Test;
+
+import junit.framework.TestCase;
+
+public class ParserFactoryTest extends TestCase {
+
+ private final ArrayBackedValueStorage storage = new ArrayBackedValueStorage();
+ private final IValueParser integerParser = IntegerParserFactory.INSTANCE.createValueParser();
+ private final IValueParser longParser = LongParserFactory.INSTANCE.createValueParser();
+ private final IValueParser booleanParser = BooleanParserFactory.INSTANCE.createValueParser();
+ private String chars = "";
+
+ @Test
+ public void testInteger() throws HyracksDataException {
+ String number = "12";
+ parse(number, integerParser, storage, IntegerPointable::getInteger, 12, true);
+ number = "+12";
+ parse(number, integerParser, storage, IntegerPointable::getInteger, 12, true);
+ number = "-12";
+ parse(number, integerParser, storage, IntegerPointable::getInteger, -12, true);
+ number = " 12";
+ parse(number, integerParser, storage, IntegerPointable::getInteger, 12, true);
+ number = " +12";
+ parse(number, integerParser, storage, IntegerPointable::getInteger, 12, true);
+ number = " -12";
+ parse(number, integerParser, storage, IntegerPointable::getInteger, -12, true);
+ number = "12 ";
+ parse(number, integerParser, storage, IntegerPointable::getInteger, 12, true);
+ number = "+12 ";
+ parse(number, integerParser, storage, IntegerPointable::getInteger, 12, true);
+ number = "-12 ";
+ parse(number, integerParser, storage, IntegerPointable::getInteger, -12, true);
+ number = " 12 ";
+ parse(number, integerParser, storage, IntegerPointable::getInteger, 12, true);
+ number = " +12 ";
+ parse(number, integerParser, storage, IntegerPointable::getInteger, 12, true);
+ number = " -12 ";
+ parse(number, integerParser, storage, IntegerPointable::getInteger, -12, true);
+
+ number = Integer.toString(Integer.MAX_VALUE);
+ parse(number, integerParser, storage, IntegerPointable::getInteger, Integer.MAX_VALUE, true);
+ number = Integer.toString(Integer.MIN_VALUE);
+ parse(number, integerParser, storage, IntegerPointable::getInteger, Integer.MIN_VALUE, true);
+
+ // overflow and underflow
+ number = Long.toString(Integer.MAX_VALUE + 1L);
+ parse(number, integerParser, storage, IntegerPointable::getInteger, null, false);
+ number = Long.toString(Integer.MIN_VALUE - 1L);
+ parse(number, integerParser, storage, IntegerPointable::getInteger, null, false);
+
+ // invalid
+ number = "a";
+ parse(number, integerParser, storage, IntegerPointable::getInteger, null, false);
+ number = "12a";
+ parse(number, integerParser, storage, IntegerPointable::getInteger, null, false);
+ number = "12 a";
+ parse(number, integerParser, storage, IntegerPointable::getInteger, null, false);
+ number = " a 12";
+ parse(number, integerParser, storage, IntegerPointable::getInteger, null, false);
+ number = "a12";
+ parse(number, integerParser, storage, IntegerPointable::getInteger, null, false);
+ number = "+ 12";
+ parse(number, integerParser, storage, IntegerPointable::getInteger, null, false);
+ number = "- 12";
+ parse(number, integerParser, storage, IntegerPointable::getInteger, null, false);
+ number = "1 2";
+ parse(number, integerParser, storage, IntegerPointable::getInteger, null, false);
+ }
+
+ @Test
+ public void testLong() throws HyracksDataException {
+ storage.reset();
+ String number = "12";
+ parse(number, longParser, storage, LongPointable::getLong, 12L, true);
+ number = "+12";
+ parse(number, longParser, storage, LongPointable::getLong, 12L, true);
+ number = "-12";
+ parse(number, longParser, storage, LongPointable::getLong, -12L, true);
+ number = " 12";
+ parse(number, longParser, storage, LongPointable::getLong, 12L, true);
+ number = " +12";
+ parse(number, longParser, storage, LongPointable::getLong, 12L, true);
+ number = " -12";
+ parse(number, longParser, storage, LongPointable::getLong, -12L, true);
+ number = "12 ";
+ parse(number, longParser, storage, LongPointable::getLong, 12L, true);
+ number = "+12 ";
+ parse(number, longParser, storage, LongPointable::getLong, 12L, true);
+ number = "-12 ";
+ parse(number, longParser, storage, LongPointable::getLong, -12L, true);
+ number = " 12 ";
+ parse(number, longParser, storage, LongPointable::getLong, 12L, true);
+ number = " +12 ";
+ parse(number, longParser, storage, LongPointable::getLong, 12L, true);
+ number = " -12 ";
+ parse(number, longParser, storage, LongPointable::getLong, -12L, true);
+
+ number = Long.toString(Long.MAX_VALUE);
+ parse(number, longParser, storage, LongPointable::getLong, Long.MAX_VALUE, true);
+ number = Long.toString(Long.MIN_VALUE);
+ parse(number, longParser, storage, LongPointable::getLong, Long.MIN_VALUE, true);
+
+ // overflow and underflow
+ number = Long.toString(Long.MAX_VALUE) + "1";
+ parse(number, longParser, storage, LongPointable::getLong, null, false);
+ number = Long.toString(Long.MIN_VALUE) + "1";
+ parse(number, longParser, storage, LongPointable::getLong, null, false);
+
+ // invalid
+ number = "a";
+ parse(number, longParser, storage, LongPointable::getLong, null, false);
+ number = "12a";
+ parse(number, longParser, storage, LongPointable::getLong, null, false);
+ number = "12 a";
+ parse(number, longParser, storage, LongPointable::getLong, null, false);
+ number = " a 12";
+ parse(number, longParser, storage, LongPointable::getLong, null, false);
+ number = "+ 12";
+ parse(number, longParser, storage, LongPointable::getLong, null, false);
+ number = "- 12";
+ parse(number, longParser, storage, LongPointable::getLong, null, false);
+ number = "1 2";
+ parse(number, longParser, storage, LongPointable::getLong, null, false);
+ }
+
+ @Test
+ public void testBoolean() throws HyracksDataException {
+ storage.reset();
+ String bool = "true";
+ parse(bool, booleanParser, storage, BooleanPointable::getBoolean, Boolean.TRUE, true);
+ bool = "TRUE";
+ parse(bool, booleanParser, storage, BooleanPointable::getBoolean, Boolean.TRUE, true);
+ bool = "True";
+ parse(bool, booleanParser, storage, BooleanPointable::getBoolean, Boolean.TRUE, true);
+ bool = "true ";
+ parse(bool, booleanParser, storage, BooleanPointable::getBoolean, Boolean.TRUE, true);
+ bool = " true";
+ parse(bool, booleanParser, storage, BooleanPointable::getBoolean, Boolean.TRUE, true);
+ bool = " True ";
+ parse(bool, booleanParser, storage, BooleanPointable::getBoolean, Boolean.TRUE, true);
+
+ bool = "false";
+ parse(bool, booleanParser, storage, BooleanPointable::getBoolean, Boolean.FALSE, true);
+ bool = "FALSE";
+ parse(bool, booleanParser, storage, BooleanPointable::getBoolean, Boolean.FALSE, true);
+ bool = "False";
+ parse(bool, booleanParser, storage, BooleanPointable::getBoolean, Boolean.FALSE, true);
+ bool = " false ";
+ parse(bool, booleanParser, storage, BooleanPointable::getBoolean, Boolean.FALSE, true);
+ bool = " false";
+ parse(bool, booleanParser, storage, BooleanPointable::getBoolean, Boolean.FALSE, true);
+ bool = "false ";
+ parse(bool, booleanParser, storage, BooleanPointable::getBoolean, Boolean.FALSE, true);
+
+ // invalid
+ bool = "foo";
+ parse(bool, booleanParser, storage, BooleanPointable::getBoolean, null, false);
+ bool = "truea";
+ parse(bool, booleanParser, storage, BooleanPointable::getBoolean, null, false);
+ bool = "ffalse";
+ parse(bool, booleanParser, storage, BooleanPointable::getBoolean, null, false);
+ bool = "ffalse";
+ parse(bool, booleanParser, storage, BooleanPointable::getBoolean, null, false);
+ bool = "t rue";
+ parse(bool, booleanParser, storage, BooleanPointable::getBoolean, null, false);
+ bool = "true a";
+ parse(bool, booleanParser, storage, BooleanPointable::getBoolean, null, false);
+ }
+
+ private <T> void parse(String test, IValueParser parser, ArrayBackedValueStorage storage, Getter<T> getter,
+ T expectedVal, boolean expectedResult) throws HyracksDataException {
+ int oldSize = storage.getLength();
+ int start = storage.getLength();
+ int stringStart = chars.length();
+ chars = chars + test;
+ int stringLength = chars.length() - stringStart;
+ boolean result = parser.parse(chars.toCharArray(), stringStart, stringLength, storage.getDataOutput());
+ int newSize = storage.getLength();
+ if (!result) {
+ assertEquals(oldSize, newSize);
+ } else {
+ assertEquals(expectedVal, getter.get(storage.getByteArray(), start));
+ }
+ assertEquals(expectedResult, result);
+
+ }
+
+ @FunctionalInterface
+ private interface Getter<T> {
+ T get(byte[] bytes, int start);
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java
index 9ddb4c2..b8b2ba8 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/file/DelimitedDataTupleParserFactory.java
@@ -28,12 +28,16 @@
import org.apache.hyracks.api.comm.VSizeFrame;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
import org.apache.hyracks.dataflow.common.data.parsers.IValueParser;
import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+/**
+ * Currently used by tests only. Reconsider the code when it will be used otherwise.
+ */
public class DelimitedDataTupleParserFactory implements ITupleParserFactory {
private static final long serialVersionUID = 1L;
private IValueParserFactory[] valueParserFactories;
@@ -54,6 +58,8 @@
@Override
public ITupleParser createTupleParser(final IHyracksTaskContext ctx) {
return new ITupleParser() {
+ final IWarningCollector warningCollector = ctx.getWarningCollector();
+
@Override
public void parse(InputStream in, IFrameWriter writer) throws HyracksDataException {
try {
@@ -67,21 +73,29 @@
ArrayTupleBuilder tb = new ArrayTupleBuilder(valueParsers.length);
DataOutput dos = tb.getDataOutput();
- FieldCursorForDelimitedDataParser cursor =
- new FieldCursorForDelimitedDataParser(new InputStreamReader(in), fieldDelimiter, quote);
+ FieldCursorForDelimitedDataParser cursor = new FieldCursorForDelimitedDataParser(
+ new InputStreamReader(in), fieldDelimiter, quote, warningCollector, () -> "");
while (cursor.nextRecord()) {
tb.reset();
for (int i = 0; i < valueParsers.length; ++i) {
- if (!cursor.nextField()) {
- break;
+ FieldCursorForDelimitedDataParser.Result result = cursor.nextField();
+ switch (result) {
+ case OK:
+ break;
+ case END:
+ case ERROR:
+ throw new HyracksDataException("Failed to parse record");
+ default:
+ throw new IllegalStateException();
}
// Eliminate double quotes in the field that we are going to parse
- if (cursor.isDoubleQuoteIncludedInThisField) {
- cursor.eliminateDoubleQuote(cursor.buffer, cursor.fStart, cursor.fEnd - cursor.fStart);
- cursor.fEnd -= cursor.doubleQuoteCount;
- cursor.isDoubleQuoteIncludedInThisField = false;
+ if (cursor.fieldHasDoubleQuote()) {
+ cursor.eliminateDoubleQuote();
}
- valueParsers[i].parse(cursor.buffer, cursor.fStart, cursor.fEnd - cursor.fStart, dos);
+ if (!valueParsers[i].parse(cursor.getBuffer(), cursor.getFieldStart(),
+ cursor.getFieldLength(), dos)) {
+ throw new HyracksDataException("Failed to parse field");
+ }
tb.addFieldEndOffset();
}
FrameUtils.appendToWriter(writer, appender, tb.getFieldEndOffsets(), tb.getByteArray(), 0,
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/file/FieldCursorForDelimitedDataParser.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/file/FieldCursorForDelimitedDataParser.java
index 7e5ee2c..e401efc 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/file/FieldCursorForDelimitedDataParser.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/file/FieldCursorForDelimitedDataParser.java
@@ -21,6 +21,11 @@
import java.io.IOException;
import java.io.Reader;
import java.util.Arrays;
+import java.util.function.Supplier;
+
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.Warning;
public class FieldCursorForDelimitedDataParser {
@@ -29,21 +34,33 @@
IN_RECORD, //cursor is inside record
EOR, //cursor is at end of record
CR, //cursor at carriage return
- EOF //end of stream reached
+ EOF, //end of stream reached
+ FAILED // cursor failed to parse a field
}
- public char[] buffer; //buffer to holds the input coming form the underlying input stream
- public int fStart; //start position for field
- public int fEnd; //end position for field
- public int recordCount; //count of records
- public int fieldCount; //count of fields in current record
- public int doubleQuoteCount; //count of double quotes
- public boolean isDoubleQuoteIncludedInThisField; //does current field include double quotes
+ public enum Result {
+ OK,
+ ERROR,
+ END
+ }
+
+ private static final String CLOSING_Q = "missing a closing quote";
+ private static final String OPENING_Q = "a quote should be in the beginning";
+ private static final String DELIMITER_AFTER_Q = "a quote enclosing a field needs to be followed by the delimiter";
+ private final IWarningCollector warnings;
+ private final Supplier<String> dataSourceName;
+ private char[] buffer; //buffer to holds the input coming form the underlying input stream
+ private int fStart; //start position for field
+ private int fEnd; //end position for field
+ private long lineCount; //count of lines
+ private int fieldCount; //count of fields in current record
+ private int doubleQuoteCount; //count of double quotes
+ private boolean isDoubleQuoteIncludedInThisField; //does current field include double quotes
private static final int INITIAL_BUFFER_SIZE = 4096;//initial buffer size
private static final int INCREMENT = 4096; //increment size
- private Reader in; //the underlying buffer
+ private final Reader in; //the underlying buffer
private int start; //start of valid buffer area
private int end; //end of valid buffer area
@@ -55,10 +72,13 @@
private int quoteCount; //count of single quotes
private boolean startedQuote; //whether a quote has been started
- private char quote; //the quote character
- private char fieldDelimiter; //the delimiter
+ private final char quote; //the quote character
+ private final char fieldDelimiter; //the delimiter
- public FieldCursorForDelimitedDataParser(Reader in, char fieldDelimiter, char quote) {
+ public FieldCursorForDelimitedDataParser(Reader in, char fieldDelimiter, char quote,
+ IWarningCollector warningCollector, Supplier<String> dataSourceName) {
+ this.warnings = warningCollector;
+ this.dataSourceName = dataSourceName;
this.in = in;
if (in != null) {
buffer = new char[INITIAL_BUFFER_SIZE];
@@ -70,20 +90,55 @@
state = State.INIT;
this.quote = quote;
this.fieldDelimiter = fieldDelimiter;
- lastDelimiterPosition = -99;
- lastQuotePosition = -99;
- lastDoubleQuotePosition = -99;
+ lastDelimiterPosition = -1;
+ lastQuotePosition = -1;
+ lastDoubleQuotePosition = -1;
quoteCount = 0;
doubleQuoteCount = 0;
startedQuote = false;
isDoubleQuoteIncludedInThisField = false;
- recordCount = 0;
+ lineCount = 1;
fieldCount = 0;
}
- public void nextRecord(char[] buffer, int recordLength) throws IOException {
- recordCount++;
+ public char[] getBuffer() {
+ return buffer;
+ }
+
+ public int getFieldStart() {
+ return fStart;
+ }
+
+ public int getFieldLength() {
+ return fEnd - fStart;
+ }
+
+ public boolean isFieldEmpty() {
+ return fStart == fEnd;
+ }
+
+ public boolean fieldHasDoubleQuote() {
+ return isDoubleQuoteIncludedInThisField;
+ }
+
+ public int getFieldCount() {
+ return fieldCount;
+ }
+
+ public long getLineCount() {
+ return lineCount;
+ }
+
+ public void nextRecord(char[] buffer, int recordLength, long lineNumber) {
+ lineCount = lineNumber;
fieldCount = 0;
+ lastDelimiterPosition = -1;
+ lastQuotePosition = -1;
+ lastDoubleQuotePosition = -1;
+ quoteCount = 0;
+ doubleQuoteCount = 0;
+ startedQuote = false;
+ isDoubleQuoteIncludedInThisField = false;
start = 0;
end = recordLength;
state = State.IN_RECORD;
@@ -91,7 +146,6 @@
}
public boolean nextRecord() throws IOException {
- recordCount++;
fieldCount = 0;
while (true) {
switch (state) {
@@ -107,6 +161,7 @@
case IN_RECORD:
int p = start;
+ char lastChar = '\0';
while (true) {
if (p >= end) {
int s = start;
@@ -147,6 +202,11 @@
lastDelimiterPosition = p;
break;
}
+ // count lines inside quotes
+ if (ch == '\r' || (ch == '\n' && lastChar != '\r')) {
+ lineCount++;
+ }
+ lastChar = ch;
++p;
}
break;
@@ -160,6 +220,10 @@
}
}
char ch = buffer[start];
+ // if the next char "ch" is not \n, then count the \r
+ if (ch != '\n') {
+ lineCount++;
+ }
if (ch == '\n' && !startedQuote) {
++start;
state = State.EOR;
@@ -169,6 +233,7 @@
}
case EOR:
+ lineCount++;
if (start >= end) {
eof = !readMore();
if (eof) {
@@ -182,62 +247,77 @@
case EOF:
return false;
+ case FAILED:
+ return false;
}
}
}
- public boolean nextField() throws IOException {
- fieldCount++;
+ public Result nextField() throws IOException {
switch (state) {
case INIT:
case EOR:
case EOF:
case CR:
- return false;
+ return Result.END;
+ case FAILED:
+ return Result.ERROR;
case IN_RECORD:
- boolean eof;
+ fieldCount++;
// reset quote related values
startedQuote = false;
isDoubleQuoteIncludedInThisField = false;
- lastQuotePosition = -99;
- lastDoubleQuotePosition = -99;
+ lastQuotePosition = -1;
+ lastDoubleQuotePosition = -1;
quoteCount = 0;
doubleQuoteCount = 0;
+ char lastChar = '\0';
int p = start;
while (true) {
if (p >= end) {
int s = start;
- eof = !readMore();
+ boolean eof = !readMore();
p -= (s - start);
- lastQuotePosition -= (s - start);
- lastDoubleQuotePosition -= (s - start);
- lastDelimiterPosition -= (s - start);
+ lastQuotePosition -= (lastQuotePosition > -1) ? (s - start) : 0;
+ lastDoubleQuotePosition -= (lastDoubleQuotePosition > -1) ? (s - start) : 0;
+ lastDelimiterPosition -= (lastDelimiterPosition > -1) ? (s - start) : 0;
if (eof) {
state = State.EOF;
- if (startedQuote && lastQuotePosition == p - 1 && lastDoubleQuotePosition != p - 1
- && quoteCount == doubleQuoteCount * 2 + 2) {
- // set the position of fStart to +1, fEnd to -1 to remove quote character
- fStart = start + 1;
- fEnd = p - 1;
- } else {
+ if (!startedQuote) {
fStart = start;
fEnd = p;
+ } else {
+ if (lastQuotePosition == p - 1 && lastDoubleQuotePosition != p - 1
+ && quoteCount == doubleQuoteCount * 2 + 2) {
+ // set the position of fStart to +1, fEnd to -1 to remove quote character
+ fStart = start + 1;
+ fEnd = p - 1;
+ } else {
+ state = State.FAILED;
+ if (warnings.shouldWarn()) {
+ warn(CLOSING_Q);
+ }
+ return Result.ERROR;
+ }
}
- return true;
+ return Result.OK;
}
}
char ch = buffer[p];
if (ch == quote) {
// If this is first quote in the field, then it needs to be placed in the beginning.
if (!startedQuote) {
- if (lastDelimiterPosition == p - 1 || lastDelimiterPosition == -99) {
+ if (p == start) {
startedQuote = true;
} else {
// In this case, we don't have a quote in the beginning of a field.
- throw new IOException("At record: " + recordCount + ", field#: " + fieldCount
- + " - a quote enclosing a field needs to be placed in the beginning of that field.");
+ state = State.FAILED;
+ if (warnings.shouldWarn()) {
+ warn(OPENING_Q);
+ }
+ return Result.ERROR;
}
}
// Check double quotes - "". We check [start != p-2]
@@ -245,8 +325,8 @@
// since it looks like a double quote. However, it's not a double quote.
// (e.g. if field2 has no value:
// field1,"",field3 ... )
- if (lastQuotePosition == p - 1 && lastDelimiterPosition != p - 2
- && lastDoubleQuotePosition != p - 1) {
+ if (lastQuotePosition == p - 1 && lastDoubleQuotePosition != p - 1
+ && lastQuotePosition != start) {
isDoubleQuoteIncludedInThisField = true;
doubleQuoteCount++;
lastDoubleQuotePosition = p;
@@ -261,76 +341,69 @@
fEnd = p;
start = p + 1;
lastDelimiterPosition = p;
- return true;
- } else if (startedQuote) {
- if (lastQuotePosition == p - 1 && lastDoubleQuotePosition != p - 1) {
- // There is a quote right before the delimiter (e.g. ",) and it is not two quote,
- // then the field contains a valid string.
- // We set the position of fStart to +1, fEnd to -1 to remove quote character
- fStart = start + 1;
- fEnd = p - 1;
- start = p + 1;
- lastDelimiterPosition = p;
- startedQuote = false;
- return true;
- } else if (lastQuotePosition < p - 1 && lastQuotePosition != lastDoubleQuotePosition
- && quoteCount == doubleQuoteCount * 2 + 2) {
- // There is a quote before the delimiter, however it is not directly placed before the delimiter.
- // In this case, we throw an exception.
- // quoteCount == doubleQuoteCount * 2 + 2 : only true when we have two quotes except double-quotes.
- throw new IOException("At record: " + recordCount + ", field#: " + fieldCount
- + " - A quote enclosing a field needs to be followed by the delimiter.");
+ return Result.OK;
+ }
+
+ if (lastQuotePosition == p - 1 && lastDoubleQuotePosition != p - 1
+ && lastQuotePosition != start) {
+ // There is a quote right before the delimiter (e.g. ",) and it is not two quote,
+ // then the field contains a valid string.
+ // We set the position of fStart to +1, fEnd to -1 to remove quote character
+ fStart = start + 1;
+ fEnd = p - 1;
+ start = p + 1;
+ lastDelimiterPosition = p;
+ startedQuote = false;
+ return Result.OK;
+ } else if (lastQuotePosition < p - 1 && lastQuotePosition != lastDoubleQuotePosition
+ && quoteCount == doubleQuoteCount * 2 + 2) {
+ // There is a quote before the delimiter, however it is not directly placed before the delimiter.
+ // In this case, we throw an exception.
+ // quoteCount == doubleQuoteCount * 2 + 2 : only true when we have two quotes except double-quotes.
+ state = State.FAILED;
+ if (warnings.shouldWarn()) {
+ warn(DELIMITER_AFTER_Q);
}
+ return Result.ERROR;
}
// If the control flow reaches here: we have a delimiter in this field and
// there should be a quote in the beginning and the end of
// this field. So, just continue reading next character
- } else if (ch == '\n') {
+ } else if (ch == '\n' || ch == '\r') {
if (!startedQuote) {
fStart = start;
fEnd = p;
start = p + 1;
- state = State.EOR;
+ state = ch == '\n' ? State.EOR : State.CR;
lastDelimiterPosition = p;
- return true;
- } else if (startedQuote && lastQuotePosition == p - 1 && lastDoubleQuotePosition != p - 1
+ return Result.OK;
+ } else if (lastQuotePosition == p - 1 && lastDoubleQuotePosition != p - 1
&& quoteCount == doubleQuoteCount * 2 + 2) {
// set the position of fStart to +1, fEnd to -1 to remove quote character
fStart = start + 1;
fEnd = p - 1;
lastDelimiterPosition = p;
start = p + 1;
- state = State.EOR;
+ state = ch == '\n' ? State.EOR : State.CR;
startedQuote = false;
- return true;
- }
- } else if (ch == '\r') {
- if (!startedQuote) {
- fStart = start;
- fEnd = p;
- start = p + 1;
- state = State.CR;
- lastDelimiterPosition = p;
- return true;
- } else if (startedQuote && lastQuotePosition == p - 1 && lastDoubleQuotePosition != p - 1
- && quoteCount == doubleQuoteCount * 2 + 2) {
- // set the position of fStart to +1, fEnd to -1 to remove quote character
- fStart = start + 1;
- fEnd = p - 1;
- lastDelimiterPosition = p;
- start = p + 1;
- state = State.CR;
- startedQuote = false;
- return true;
+ return Result.OK;
}
}
+ // count lines inside quotes
+ if (ch == '\r' || (ch == '\n' && lastChar != '\r')) {
+ lineCount++;
+ }
+ lastChar = ch;
++p;
}
}
throw new IllegalStateException();
}
- protected boolean readMore() throws IOException {
+ private boolean readMore() throws IOException {
+ if (in == null) {
+ return false;
+ }
if (start > 0) {
System.arraycopy(buffer, start, buffer, 0, end - start);
}
@@ -350,10 +423,11 @@
}
// Eliminate escaped double quotes("") in a field
- public void eliminateDoubleQuote(char[] buffer, int start, int length) {
- int lastDoubleQuotePosition = -99;
- int writepos = start;
- int readpos = start;
+ public void eliminateDoubleQuote() {
+ int lastDoubleQuotePosition = -1;
+ int writepos = fStart;
+ int readpos = fStart;
+ int length = fEnd - fStart;
// Find positions where double quotes appear
for (int i = 0; i < length; i++) {
// Skip double quotes
@@ -369,5 +443,12 @@
readpos++;
}
}
+ fEnd -= doubleQuoteCount;
+ isDoubleQuoteIncludedInThisField = false;
+ }
+
+ private void warn(String message) {
+ warnings.warn(Warning.forHyracks(null, ErrorCode.PARSING_ERROR, dataSourceName.get(), lineCount, fieldCount,
+ message));
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NestedLoopJoin.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NestedLoopJoin.java
index 361d1ee..dbc91c2 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NestedLoopJoin.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NestedLoopJoin.java
@@ -20,6 +20,7 @@
import java.io.DataOutput;
import java.nio.ByteBuffer;
+import java.util.BitSet;
import org.apache.hyracks.api.comm.IFrame;
import org.apache.hyracks.api.comm.IFrameWriter;
@@ -28,6 +29,7 @@
import org.apache.hyracks.api.dataflow.value.IMissingWriter;
import org.apache.hyracks.api.dataflow.value.IPredicateEvaluator;
import org.apache.hyracks.api.dataflow.value.ITuplePairComparator;
+import org.apache.hyracks.api.exceptions.ErrorCode;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.FileReference;
import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
@@ -43,6 +45,14 @@
import org.apache.hyracks.dataflow.std.buffermanager.VariableFramePool;
public class NestedLoopJoin {
+ // Note: Min memory budget should be less than {@code AbstractJoinPOperator.MIN_FRAME_LIMIT_FOR_JOIN}
+ // Inner join: 1 frame for the outer input side, 1 frame for the inner input side, 1 frame for the output
+ private static final int MIN_FRAME_BUDGET_INNER_JOIN = 3;
+ // Outer join extra: Add 1 frame for the {@code outerMatchLOJ} bitset
+ private static final int MIN_FRAME_BUDGET_OUTER_JOIN = MIN_FRAME_BUDGET_INNER_JOIN + 1;
+ // Outer join needs 1 bit per each tuple in the outer side buffer
+ private static final int ESTIMATE_AVG_TUPLE_SIZE = 128;
+
private final FrameTupleAccessor accessorInner;
private final FrameTupleAccessor accessorOuter;
private final FrameTupleAppender appender;
@@ -54,30 +64,45 @@
private final boolean isLeftOuter;
private final ArrayTupleBuilder missingTupleBuilder;
private final IPredicateEvaluator predEvaluator;
- private boolean isReversed; //Added for handling correct calling for predicate-evaluator upon recursive calls (in OptimizedHybridHashJoin) that cause role-reversal
+ // Added for handling correct calling for predicate-evaluator upon recursive calls
+ // (in OptimizedHybridHashJoin) that cause role-reversal
+ private final boolean isReversed;
private final BufferInfo tempInfo = new BufferInfo(null, -1, -1);
+ private final BitSet outerMatchLOJ;
public NestedLoopJoin(IHyracksJobletContext jobletContext, FrameTupleAccessor accessorOuter,
- FrameTupleAccessor accessorInner, int memSize, IPredicateEvaluator predEval, boolean isLeftOuter,
+ FrameTupleAccessor accessorInner, int memBudgetInFrames, IPredicateEvaluator predEval, boolean isLeftOuter,
IMissingWriter[] missingWriters) throws HyracksDataException {
+ this(jobletContext, accessorOuter, accessorInner, memBudgetInFrames, predEval, isLeftOuter, missingWriters,
+ false);
+ }
+
+ public NestedLoopJoin(IHyracksJobletContext jobletContext, FrameTupleAccessor accessorOuter,
+ FrameTupleAccessor accessorInner, int memBudgetInFrames, IPredicateEvaluator predEval, boolean isLeftOuter,
+ IMissingWriter[] missingWriters, boolean isReversed) throws HyracksDataException {
this.accessorInner = accessorInner;
this.accessorOuter = accessorOuter;
this.appender = new FrameTupleAppender();
this.outBuffer = new VSizeFrame(jobletContext);
this.innerBuffer = new VSizeFrame(jobletContext);
this.appender.reset(outBuffer, true);
- if (memSize < 3) {
- throw new HyracksDataException("Not enough memory is available for Nested Loop Join");
+
+ int minMemBudgetInFrames = isLeftOuter ? MIN_FRAME_BUDGET_OUTER_JOIN : MIN_FRAME_BUDGET_INNER_JOIN;
+ if (memBudgetInFrames < minMemBudgetInFrames) {
+ throw new HyracksDataException(ErrorCode.INSUFFICIENT_MEMORY);
}
+ int outerBufferMngrMemBudgetInFrames = memBudgetInFrames - minMemBudgetInFrames + 1;
+ int outerBufferMngrMemBudgetInBytes = jobletContext.getInitialFrameSize() * outerBufferMngrMemBudgetInFrames;
this.outerBufferMngr = new VariableFrameMemoryManager(
- new VariableFramePool(jobletContext, jobletContext.getInitialFrameSize() * (memSize - 2)),
- FrameFreeSlotPolicyFactory.createFreeSlotPolicy(EnumFreeSlotPolicy.LAST_FIT, memSize - 2));
+ new VariableFramePool(jobletContext, outerBufferMngrMemBudgetInBytes), FrameFreeSlotPolicyFactory
+ .createFreeSlotPolicy(EnumFreeSlotPolicy.LAST_FIT, outerBufferMngrMemBudgetInFrames));
this.predEvaluator = predEval;
- this.isReversed = false;
-
this.isLeftOuter = isLeftOuter;
if (isLeftOuter) {
+ if (isReversed) {
+ throw new HyracksDataException(ErrorCode.INVALID_OPERATOR_OPERATION, "reverse", "outer join");
+ }
int innerFieldCount = this.accessorInner.getFieldCount();
missingTupleBuilder = new ArrayTupleBuilder(innerFieldCount);
DataOutput out = missingTupleBuilder.getDataOutput();
@@ -85,9 +110,14 @@
missingWriters[i].writeMissing(out);
missingTupleBuilder.addFieldEndOffset();
}
+ // Outer join needs 1 bit per each tuple in the outer side buffer
+ int outerMatchLOJCardinalityEstimate = outerBufferMngrMemBudgetInBytes / ESTIMATE_AVG_TUPLE_SIZE;
+ outerMatchLOJ = new BitSet(Math.max(outerMatchLOJCardinalityEstimate, 1));
} else {
missingTupleBuilder = null;
+ outerMatchLOJ = null;
}
+ this.isReversed = isReversed;
FileReference file =
jobletContext.createManagedWorkspaceFile(this.getClass().getSimpleName() + this.toString());
@@ -110,17 +140,7 @@
public void join(ByteBuffer outerBuffer, IFrameWriter writer) throws HyracksDataException {
if (outerBufferMngr.insertFrame(outerBuffer) < 0) {
- RunFileReader runFileReader = runFileWriter.createReader();
- try {
- runFileReader.open();
- while (runFileReader.nextFrame(innerBuffer)) {
- for (int i = 0; i < outerBufferMngr.getNumFrames(); i++) {
- blockJoin(outerBufferMngr.getFrame(i, tempInfo), innerBuffer.getBuffer(), writer);
- }
- }
- } finally {
- runFileReader.close();
- }
+ multiBlockJoin(writer);
outerBufferMngr.reset();
if (outerBufferMngr.insertFrame(outerBuffer) < 0) {
throw new HyracksDataException("The given outer frame of size:" + outerBuffer.capacity()
@@ -129,16 +149,51 @@
}
}
- private void blockJoin(BufferInfo outerBufferInfo, ByteBuffer innerBuffer, IFrameWriter writer)
- throws HyracksDataException {
- accessorOuter.reset(outerBufferInfo.getBuffer(), outerBufferInfo.getStartOffset(), outerBufferInfo.getLength());
- accessorInner.reset(innerBuffer);
- int tupleCount0 = accessorOuter.getTupleCount();
- int tupleCount1 = accessorInner.getTupleCount();
+ private void multiBlockJoin(IFrameWriter writer) throws HyracksDataException {
+ int outerBufferFrameCount = outerBufferMngr.getNumFrames();
+ if (outerBufferFrameCount == 0) {
+ return;
+ }
+ RunFileReader runFileReader = runFileWriter.createReader();
+ try {
+ runFileReader.open();
+ if (isLeftOuter) {
+ outerMatchLOJ.clear();
+ }
+ while (runFileReader.nextFrame(innerBuffer)) {
+ int outerTupleRunningCount = 0;
+ for (int i = 0; i < outerBufferFrameCount; i++) {
+ BufferInfo outerBufferInfo = outerBufferMngr.getFrame(i, tempInfo);
+ accessorOuter.reset(outerBufferInfo.getBuffer(), outerBufferInfo.getStartOffset(),
+ outerBufferInfo.getLength());
+ int outerTupleCount = accessorOuter.getTupleCount();
+ accessorInner.reset(innerBuffer.getBuffer());
+ blockJoin(outerTupleRunningCount, writer);
+ outerTupleRunningCount += outerTupleCount;
+ }
+ }
+ if (isLeftOuter) {
+ int outerTupleRunningCount = 0;
+ for (int i = 0; i < outerBufferFrameCount; i++) {
+ BufferInfo outerBufferInfo = outerBufferMngr.getFrame(i, tempInfo);
+ accessorOuter.reset(outerBufferInfo.getBuffer(), outerBufferInfo.getStartOffset(),
+ outerBufferInfo.getLength());
+ int outerFrameTupleCount = accessorOuter.getTupleCount();
+ appendMissing(outerTupleRunningCount, outerFrameTupleCount, writer);
+ outerTupleRunningCount += outerFrameTupleCount;
+ }
+ }
+ } finally {
+ runFileReader.close();
+ }
+ }
- for (int i = 0; i < tupleCount0; ++i) {
+ private void blockJoin(int outerTupleStartPos, IFrameWriter writer) throws HyracksDataException {
+ int outerTupleCount = accessorOuter.getTupleCount();
+ int innerTupleCount = accessorInner.getTupleCount();
+ for (int i = 0; i < outerTupleCount; ++i) {
boolean matchFound = false;
- for (int j = 0; j < tupleCount1; ++j) {
+ for (int j = 0; j < innerTupleCount; ++j) {
int c = tpComparator.compare(accessorOuter, i, accessorInner, j);
boolean prdEval = evaluatePredicate(i, j);
if (c == 0 && prdEval) {
@@ -146,13 +201,8 @@
appendToResults(i, j, writer);
}
}
-
- if (!matchFound && isLeftOuter) {
- final int[] ntFieldEndOffsets = missingTupleBuilder.getFieldEndOffsets();
- final byte[] ntByteArray = missingTupleBuilder.getByteArray();
- final int ntSize = missingTupleBuilder.getSize();
- FrameUtils.appendConcatToWriter(writer, appender, accessorOuter, i, ntFieldEndOffsets, ntByteArray, 0,
- ntSize);
+ if (isLeftOuter && matchFound) {
+ outerMatchLOJ.set(outerTupleStartPos + i);
}
}
}
@@ -178,6 +228,21 @@
FrameUtils.appendConcatToWriter(writer, appender, accessor1, tupleId1, accessor2, tupleId2);
}
+ private void appendMissing(int outerFrameMngrStartPos, int outerFrameTupleCount, IFrameWriter writer)
+ throws HyracksDataException {
+ int limit = outerFrameMngrStartPos + outerFrameTupleCount;
+ for (int outerTuplePos =
+ outerMatchLOJ.nextClearBit(outerFrameMngrStartPos); outerTuplePos < limit; outerTuplePos =
+ outerMatchLOJ.nextClearBit(outerTuplePos + 1)) {
+ int[] ntFieldEndOffsets = missingTupleBuilder.getFieldEndOffsets();
+ byte[] ntByteArray = missingTupleBuilder.getByteArray();
+ int ntSize = missingTupleBuilder.getSize();
+ int outerAccessorTupleIndex = outerTuplePos - outerFrameMngrStartPos;
+ FrameUtils.appendConcatToWriter(writer, appender, accessorOuter, outerAccessorTupleIndex, ntFieldEndOffsets,
+ ntByteArray, 0, ntSize);
+ }
+ }
+
public void closeCache() throws HyracksDataException {
if (runFileWriter != null) {
runFileWriter.close();
@@ -185,16 +250,10 @@
}
public void completeJoin(IFrameWriter writer) throws HyracksDataException {
- RunFileReader runFileReader = runFileWriter.createDeleteOnCloseReader();
try {
- runFileReader.open();
- while (runFileReader.nextFrame(innerBuffer)) {
- for (int i = 0; i < outerBufferMngr.getNumFrames(); i++) {
- blockJoin(outerBufferMngr.getFrame(i, tempInfo), innerBuffer.getBuffer(), writer);
- }
- }
+ multiBlockJoin(writer);
} finally {
- runFileReader.close();
+ runFileWriter.eraseClosed();
}
appender.write(writer, true);
}
@@ -202,8 +261,4 @@
public void releaseMemory() throws HyracksDataException {
outerBufferMngr.reset();
}
-
- public void setIsReversed(boolean b) {
- this.isReversed = b;
- }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
index 68b4b7b..59bb7ba 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
@@ -34,6 +34,7 @@
import org.apache.hyracks.api.exceptions.ErrorCode;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.api.util.CleanupUtils;
import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
import org.apache.hyracks.dataflow.common.io.RunFileReader;
@@ -219,6 +220,19 @@
}
}
+ public void fail() throws HyracksDataException {
+ for (RunFileWriter writer : buildRFWriters) {
+ if (writer != null) {
+ CleanupUtils.fail(writer, null);
+ }
+ }
+ for (RunFileWriter writer : probeRFWriters) {
+ if (writer != null) {
+ CleanupUtils.fail(writer, null);
+ }
+ }
+ }
+
private void closeAllSpilledPartitions(RunFileWriter[] runFileWriters, String refName) throws HyracksDataException {
try {
for (int pid = spilledStatus.nextSetBit(0); pid >= 0 && pid < numOfPartitions; pid =
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
index 1819b8d..11d2c5a 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
@@ -273,7 +273,7 @@
ITuplePartitionComputer buildHpc =
new FieldHashPartitionComputerFamily(buildKeys, buildHashFunctionFactories)
.createPartitioner(0);
- boolean isFailed = false;
+ boolean failed = false;
@Override
public void open() throws HyracksDataException {
@@ -302,21 +302,24 @@
@Override
public void close() throws HyracksDataException {
if (state.hybridHJ != null) {
- state.hybridHJ.closeBuild();
- if (isFailed) {
- state.hybridHJ.clearBuildTempFiles();
- } else {
+ if (!failed) {
+ state.hybridHJ.closeBuild();
ctx.setStateObject(state);
if (LOGGER.isTraceEnabled()) {
LOGGER.trace("OptimizedHybridHashJoin closed its build phase");
}
+ } else {
+ state.hybridHJ.clearBuildTempFiles();
}
}
}
@Override
public void fail() throws HyracksDataException {
- isFailed = true;
+ failed = true;
+ if (state.hybridHJ != null) {
+ state.hybridHJ.fail();
+ }
}
@Override
@@ -401,6 +404,9 @@
@Override
public void fail() throws HyracksDataException {
failed = true;
+ if (state.hybridHJ != null) {
+ state.hybridHJ.fail();
+ }
writer.fail();
}
@@ -447,6 +453,9 @@
joinPartitionPair(bReader, pReader, bSize, pSize, 1);
}
} catch (Exception e) {
+ if (state.hybridHJ != null) {
+ state.hybridHJ.fail();
+ }
// Since writer.nextFrame() is called in the above "try" body, we have to call writer.fail()
// to send the failure signal to the downstream, when there is a throwable thrown.
writer.fail();
@@ -786,11 +795,10 @@
// The nested loop join result is outer + inner. All the other operator is probe + build.
// Hence the reverse relation is different.
boolean isReversed = outerRd == buildRd && innerRd == probeRd;
- assert isLeftOuter ? !isReversed : true : "LeftOut Join can not reverse roles";
ITuplePairComparator nljComptorOuterInner = isReversed ? buildComp : probComp;
NestedLoopJoin nlj = new NestedLoopJoin(jobletCtx, new FrameTupleAccessor(outerRd),
- new FrameTupleAccessor(innerRd), memorySize, predEvaluator, isLeftOuter, nonMatchWriter);
- nlj.setIsReversed(isReversed);
+ new FrameTupleAccessor(innerRd), memorySize, predEvaluator, isLeftOuter, nonMatchWriter,
+ isReversed);
nlj.setComparator(nljComptorOuterInner);
IFrame cacheBuff = new VSizeFrame(jobletCtx);
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/file/CursorTest.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/file/CursorTest.java
index e663179..5561ad1 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/file/CursorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/file/CursorTest.java
@@ -30,6 +30,7 @@
public class CursorTest {
// @Test commented out due to ASTERIXDB-1881
+ // fix the code if it is to be enabled
public void test() {
FileInputStream in = null;
BufferedReader reader = null;
@@ -38,25 +39,28 @@
Paths.get(getClass().getResource("/data/beer.txt").toURI()).toAbsolutePath().toString());
reader = new BufferedReader(new InputStreamReader(in, StandardCharsets.UTF_8));
// skip header
- final FieldCursorForDelimitedDataParser cursor = new FieldCursorForDelimitedDataParser(reader, ',', '"');
+ final FieldCursorForDelimitedDataParser cursor =
+ new FieldCursorForDelimitedDataParser(reader, ',', '"', null, () -> "");
// get number of fields from header (first record is header)
cursor.nextRecord();
int numOfFields = 0;
int expectedNumberOfRecords = 7307;
- while (cursor.nextField()) {
+ FieldCursorForDelimitedDataParser.Result lastResult = cursor.nextField();
+ while ((lastResult = cursor.nextField()) == FieldCursorForDelimitedDataParser.Result.OK) {
numOfFields++;
}
+ Assert.assertNotEquals(lastResult, FieldCursorForDelimitedDataParser.Result.ERROR);
int recordNumber = 0;
while (cursor.nextRecord()) {
int fieldNumber = 0;
- while (cursor.nextField()) {
- if (cursor.isDoubleQuoteIncludedInThisField) {
- cursor.eliminateDoubleQuote(cursor.buffer, cursor.fStart, cursor.fEnd - cursor.fStart);
- cursor.fEnd -= cursor.doubleQuoteCount;
+ while ((lastResult = cursor.nextField()) == FieldCursorForDelimitedDataParser.Result.OK) {
+ if (cursor.fieldHasDoubleQuote()) {
+ cursor.eliminateDoubleQuote();
}
fieldNumber++;
}
+ Assert.assertNotEquals(lastResult, FieldCursorForDelimitedDataParser.Result.ERROR);
if ((fieldNumber > numOfFields) || (fieldNumber < numOfFields)) {
System.err.println("Test case failed. Expected number of fields in each record is " + numOfFields
+ " and record number " + recordNumber + " was found to have " + fieldNumber);
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
index 100c93a..99aa399 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
@@ -116,7 +116,7 @@
</dependency>
<dependency>
<groupId>org.mockito</groupId>
- <artifactId>mockito-all</artifactId>
+ <artifactId>mockito-core</artifactId>
<scope>test</scope>
</dependency>
<dependency>
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CancelJobTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CancelJobTest.java
index c4f2ac6..dc5d017 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CancelJobTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CancelJobTest.java
@@ -155,7 +155,7 @@
exceptionMatched = true;
Assert.assertTrue(e instanceof HyracksException);
HyracksException hyracksException = (HyracksException) e;
- Assert.assertTrue(hyracksException.getErrorCode() == ErrorCode.JOB_CANCELED);
+ Assert.assertTrue(hyracksException.matches(ErrorCode.JOB_CANCELED));
} finally {
Assert.assertTrue(exceptionMatched);
}
@@ -170,7 +170,7 @@
waitForCompletion(jobId);
} catch (HyracksException e) {
exceptionMatched = true;
- Assert.assertTrue(e.getErrorCode() == ErrorCode.JOB_CANCELED);
+ Assert.assertTrue(e.matches(ErrorCode.JOB_CANCELED));
} finally {
Assert.assertTrue(exceptionMatched);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/text-example/texthelper/src/main/java/org/apache/hyracks/examples/text/WordTupleParserFactory.java b/hyracks-fullstack/hyracks/hyracks-examples/text-example/texthelper/src/main/java/org/apache/hyracks/examples/text/WordTupleParserFactory.java
index 2219a14..0032c40 100644
--- a/hyracks-fullstack/hyracks/hyracks-examples/text-example/texthelper/src/main/java/org/apache/hyracks/examples/text/WordTupleParserFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-examples/text-example/texthelper/src/main/java/org/apache/hyracks/examples/text/WordTupleParserFactory.java
@@ -54,7 +54,9 @@
WordCursor cursor = new WordCursor(new InputStreamReader(in));
while (cursor.nextWord()) {
tb.reset();
- utf8StringParser.parse(cursor.buffer, cursor.fStart, cursor.fEnd - cursor.fStart, dos);
+ if (!utf8StringParser.parse(cursor.buffer, cursor.fStart, cursor.fEnd - cursor.fStart, dos)) {
+ throw new HyracksDataException("Failed to parse word");
+ }
tb.addFieldEndOffset();
FrameUtils.appendToWriter(writer, appender, tb.getFieldEndOffsets(), tb.getByteArray(), 0,
tb.getSize());
diff --git a/hyracks-fullstack/hyracks/hyracks-hdfs/pom.xml b/hyracks-fullstack/hyracks/hyracks-hdfs/pom.xml
index 9528eaa..a35f816 100644
--- a/hyracks-fullstack/hyracks/hyracks-hdfs/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-hdfs/pom.xml
@@ -132,6 +132,12 @@
<type>test-jar</type>
<classifier>tests</classifier>
<scope>test</scope>
+ <exclusions>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-all</artifactId>
+ </exclusion>
+ </exclusions>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
diff --git a/hyracks-fullstack/hyracks/hyracks-http/pom.xml b/hyracks-fullstack/hyracks/hyracks-http/pom.xml
index 4b6bdef..a1dd1e9 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-http/pom.xml
@@ -109,7 +109,7 @@
</dependency>
<dependency>
<groupId>org.mockito</groupId>
- <artifactId>mockito-all</artifactId>
+ <artifactId>mockito-core</artifactId>
<scope>test</scope>
</dependency>
<dependency>
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/AbstractServlet.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/AbstractServlet.java
index 21653c8..3977207 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/AbstractServlet.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/AbstractServlet.java
@@ -22,6 +22,7 @@
import static com.fasterxml.jackson.databind.SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS;
import java.io.IOException;
+import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.concurrent.ConcurrentMap;
@@ -113,7 +114,7 @@
protected void sendError(IServletResponse response, HttpResponseStatus status, String message) throws IOException {
response.setStatus(status);
- HttpUtil.setContentType(response, HttpUtil.ContentType.TEXT_PLAIN, HttpUtil.Encoding.UTF8);
+ HttpUtil.setContentType(response, HttpUtil.ContentType.TEXT_PLAIN, StandardCharsets.UTF_8);
if (message != null) {
response.writer().println(message);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/utils/HttpUtil.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/utils/HttpUtil.java
index 8451898..1f0d3bc 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/utils/HttpUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/utils/HttpUtil.java
@@ -19,6 +19,8 @@
package org.apache.hyracks.http.server.utils;
import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Reader;
import java.nio.charset.Charset;
import java.nio.charset.StandardCharsets;
import java.util.HashMap;
@@ -27,14 +29,22 @@
import java.util.Objects;
import java.util.Optional;
import java.util.OptionalDouble;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Supplier;
import java.util.regex.Pattern;
import java.util.stream.Stream;
import org.apache.commons.lang3.StringUtils;
+import org.apache.http.client.methods.CloseableHttpResponse;
import org.apache.hyracks.http.api.IServletRequest;
import org.apache.hyracks.http.api.IServletResponse;
import org.apache.hyracks.http.server.BaseRequest;
import org.apache.hyracks.http.server.FormUrlEncodedRequest;
+import org.apache.hyracks.util.ThrowingConsumer;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
@@ -43,6 +53,7 @@
import io.netty.handler.codec.http.FullHttpRequest;
import io.netty.handler.codec.http.HttpHeaderNames;
import io.netty.handler.codec.http.HttpHeaderValues;
+import io.netty.handler.codec.http.HttpMethod;
import io.netty.handler.codec.http.HttpRequest;
import io.netty.handler.codec.http.HttpScheme;
import io.netty.util.AsciiString;
@@ -66,7 +77,8 @@
public static IServletRequest toServletRequest(ChannelHandlerContext ctx, FullHttpRequest request,
HttpScheme scheme) {
return ContentType.APPLICATION_X_WWW_FORM_URLENCODED.equals(getContentTypeOnly(request))
- ? FormUrlEncodedRequest.create(ctx, request, scheme) : BaseRequest.create(ctx, request, scheme);
+ && !HttpMethod.GET.equals(request.method()) ? FormUrlEncodedRequest.create(ctx, request, scheme)
+ : BaseRequest.create(ctx, request, scheme);
}
public static String getContentTypeOnly(IServletRequest request) {
@@ -90,12 +102,12 @@
public static Charset setContentType(IServletResponse response, String type, IServletRequest fromRequest)
throws IOException {
Charset preferredCharset = getPreferredCharset(fromRequest);
- response.setHeader(HttpHeaderNames.CONTENT_TYPE, type + "; charset=" + preferredCharset.name());
+ setContentType(response, type, preferredCharset);
return preferredCharset;
}
- public static void setContentType(IServletResponse response, String type, String charset) throws IOException {
- response.setHeader(HttpHeaderNames.CONTENT_TYPE, type + "; charset=" + charset);
+ public static void setContentType(IServletResponse response, String type, Charset charset) throws IOException {
+ response.setHeader(HttpHeaderNames.CONTENT_TYPE, type + "; charset=" + charset.name());
}
public static void setContentType(IServletResponse response, String type) throws IOException {
@@ -184,10 +196,41 @@
return i < 0 ? uri : uri.substring(0, i);
}
- public static class Encoding {
- public static final String UTF8 = "utf-8";
+ public static void handleStreamInterruptibly(CloseableHttpResponse response,
+ ThrowingConsumer<Reader> streamProcessor, ExecutorService executor, Supplier<String> taskDescription)
+ throws IOException, InterruptedException, ExecutionException {
+ // we have to consume the stream in a separate thread, as it not stop on interrupt; we need to
+ // instead close the connection to achieve the interrupt
+ Future<Void> readFuture = executor.submit(() -> {
+ InputStreamReader reader = new InputStreamReader(response.getEntity().getContent(), StandardCharsets.UTF_8);
+ streamProcessor.process(new Reader() {
+ @Override
+ public int read(char[] cbuf, int off, int len) throws IOException {
+ return reader.read(cbuf, off, len);
+ }
- private Encoding() {
+ @Override
+ public void close() throws IOException {
+ // this will block until the response is closed, which will cause hangs if the stream processor
+ // tries to close the reader e.g. on processing failure
+ LOGGER.debug("ignoring close on {}", reader);
+ }
+ });
+ return null;
+ });
+ try {
+ readFuture.get();
+ } catch (InterruptedException ex) { // NOSONAR -- interrupt or rethrow
+ response.close();
+ try {
+ readFuture.get(1, TimeUnit.SECONDS);
+ } catch (TimeoutException te) {
+ LOGGER.warn("{} did not exit on stream close due to interrupt after 1s", taskDescription);
+ readFuture.cancel(true);
+ } catch (ExecutionException ee) {
+ LOGGER.debug("ignoring exception awaiting aborted {} shutdown", taskDescription, ee);
+ }
+ throw ex;
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/test/http/HttpTestUtil.java b/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/test/http/HttpTestUtil.java
index af5e3e3..4debe7f 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/test/http/HttpTestUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/test/http/HttpTestUtil.java
@@ -21,16 +21,16 @@
import java.lang.management.ManagementFactory;
import java.lang.management.MemoryPoolMXBean;
import java.lang.management.MemoryType;
-import java.lang.reflect.Field;
import java.util.List;
-import java.util.concurrent.atomic.AtomicLong;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.ByteBufAllocatorMetric;
+import io.netty.buffer.ByteBufAllocatorMetricProvider;
import io.netty.buffer.PooledByteBufAllocator;
-import io.netty.util.internal.PlatformDependent;
public class HttpTestUtil {
private static final Logger LOGGER = LogManager.getLogger();
@@ -66,14 +66,12 @@
report.append("---------------------------- Netty ----------------------------");
report.append('\n');
try {
- Field field = PlatformDependent.class.getDeclaredField("DIRECT_MEMORY_COUNTER");
- field.setAccessible(true);
- AtomicLong usedDirectMemory = (AtomicLong) field.get(null);
- long used = usedDirectMemory.get();
- report.append("Current PlatformDependent.DIRECT_MEMORY_COUNTER: ");
+ ByteBufAllocatorMetric memUsage = ((ByteBufAllocatorMetricProvider) ByteBufAllocator.DEFAULT).metric();
+ long used = memUsage.usedDirectMemory();
+ report.append("Current default allocator direct buffer usage: ");
report.append(used);
report.append('\n');
- report.append("Maximum PlatformDependent.DIRECT_MEMORY_COUNTER: ");
+ report.append("Maximum default allocator direct buffer usage: ");
maxMemUsage = Math.max(maxMemUsage, used);
report.append(maxMemUsage);
report.append('\n');
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java
index eaae8e7..5834f26 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/impl/IPCConnectionManager.java
@@ -477,8 +477,7 @@
final Object attachment = key.attachment();
if (attachment != null) {
final IPCHandle handle = (IPCHandle) attachment;
- handle.close();
- unregisterHandle(handle);
+ closeHandle(handle);
}
key.cancel();
}
@@ -527,6 +526,7 @@
connectionEstablished(handle, channelKey, socketChannel);
}
} else {
+ closeHandle(handle);
close(channelKey, socketChannel.getSocketChannel());
}
}
@@ -538,5 +538,12 @@
handle.setKey(channelKey);
channelKey.attach(handle);
}
+
+ private void closeHandle(IPCHandle handle) {
+ if (handle != null) {
+ handle.close();
+ unregisterHandle(handle);
+ }
+ }
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/security/NetworkSecurityConfig.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/security/NetworkSecurityConfig.java
index 5ebd282..770cbeb 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/security/NetworkSecurityConfig.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/security/NetworkSecurityConfig.java
@@ -20,12 +20,12 @@
import java.io.File;
import java.security.KeyStore;
-import java.util.Optional;
import org.apache.hyracks.api.network.INetworkSecurityConfig;
public class NetworkSecurityConfig implements INetworkSecurityConfig {
+ private static final long serialVersionUID = -1914030130038989199L;
private final boolean sslEnabled;
private final File keyStoreFile;
private final File trustStoreFile;
@@ -73,9 +73,4 @@
public File getTrustStoreFile() {
return trustStoreFile;
}
-
- @Override
- public Optional<String> getTrustStorePassword() {
- return Optional.empty();
- }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/security/NetworkSecurityManager.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/security/NetworkSecurityManager.java
index e352260..42dacf5 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/security/NetworkSecurityManager.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/security/NetworkSecurityManager.java
@@ -37,41 +37,16 @@
private volatile INetworkSecurityConfig config;
private final ISocketChannelFactory sslSocketFactory;
- private static final String TSL_VERSION = "TLSv1.2";
+ public static final String TSL_VERSION = "TLSv1.2";
public NetworkSecurityManager(INetworkSecurityConfig config) {
this.config = config;
- if (config.isSslEnabled()) {
- System.setProperty("javax.net.ssl.trustStore", config.getTrustStoreFile().getAbsolutePath());
- config.getTrustStorePassword().ifPresent(pw -> System.setProperty("javax.net.ssl.trustStorePassword", pw));
- }
sslSocketFactory = new SslSocketChannelFactory(this);
}
@Override
public SSLContext newSSLContext() {
- try {
- final char[] password = getKeyStorePassword();
- KeyStore engineKeyStore = config.getKeyStore();
- if (engineKeyStore == null) {
- engineKeyStore = loadKeyStoreFromFile(password);
- }
- final String defaultAlgorithm = KeyManagerFactory.getDefaultAlgorithm();
- KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(defaultAlgorithm);
- TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance(defaultAlgorithm);
- keyManagerFactory.init(engineKeyStore, password);
- KeyStore trustStore = config.getTrustStore();
- if (trustStore == null) {
- trustStore =
- loadTrustStoreFromFile(config.getTrustStorePassword().map(String::toCharArray).orElse(null));
- }
- trustManagerFactory.init(trustStore);
- SSLContext ctx = SSLContext.getInstance(TSL_VERSION);
- ctx.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), new SecureRandom());
- return ctx;
- } catch (Exception ex) {
- throw new IllegalStateException("Failed to create SSLEngine", ex);
- }
+ return newSSLContext(config);
}
@Override
@@ -101,7 +76,31 @@
this.config = config;
}
- private KeyStore loadKeyStoreFromFile(char[] password) {
+ public static SSLContext newSSLContext(INetworkSecurityConfig config) {
+ try {
+ final char[] password = getKeyStorePassword(config);
+ KeyStore engineKeyStore = config.getKeyStore();
+ if (engineKeyStore == null) {
+ engineKeyStore = loadKeyStoreFromFile(password, config);
+ }
+ final String defaultAlgorithm = KeyManagerFactory.getDefaultAlgorithm();
+ KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(defaultAlgorithm);
+ TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance(defaultAlgorithm);
+ keyManagerFactory.init(engineKeyStore, password);
+ KeyStore trustStore = config.getTrustStore();
+ if (trustStore == null) {
+ trustStore = loadTrustStoreFromFile(password, config);
+ }
+ trustManagerFactory.init(trustStore);
+ SSLContext ctx = SSLContext.getInstance(TSL_VERSION);
+ ctx.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), new SecureRandom());
+ return ctx;
+ } catch (Exception ex) {
+ throw new IllegalStateException("Failed to create SSLEngine", ex);
+ }
+ }
+
+ private static KeyStore loadKeyStoreFromFile(char[] password, INetworkSecurityConfig config) {
try {
final KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
ks.load(new FileInputStream(config.getKeyStoreFile()), password);
@@ -111,7 +110,7 @@
}
}
- private KeyStore loadTrustStoreFromFile(char[] password) {
+ private static KeyStore loadTrustStoreFromFile(char[] password, INetworkSecurityConfig config) {
try {
final KeyStore ks = KeyStore.getInstance(KeyStore.getDefaultType());
ks.load(new FileInputStream(config.getTrustStoreFile()), password);
@@ -121,7 +120,7 @@
}
}
- private char[] getKeyStorePassword() {
+ private static char[] getKeyStorePassword(INetworkSecurityConfig config) {
final String pass = config.getKeyStorePassword();
return pass == null || pass.isEmpty() ? null : pass.toCharArray();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/sockets/SslSocketChannel.java b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/sockets/SslSocketChannel.java
index 02a1a02..f9bf5c7 100644
--- a/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/sockets/SslSocketChannel.java
+++ b/hyracks-fullstack/hyracks/hyracks-ipc/src/main/java/org/apache/hyracks/ipc/sockets/SslSocketChannel.java
@@ -20,6 +20,7 @@
import java.io.IOException;
import java.nio.ByteBuffer;
+import java.nio.channels.ClosedChannelException;
import java.nio.channels.SocketChannel;
import javax.net.ssl.SSLEngine;
@@ -142,6 +143,9 @@
while (src.hasRemaining()) {
// chunk src to encrypted ssl records of pocket size
outEncryptedData.clear();
+ if (!socketChannel.isConnected()) {
+ throw new ClosedChannelException();
+ }
final SSLEngineResult result = engine.wrap(src, outEncryptedData);
switch (result.getStatus()) {
case OK:
@@ -186,8 +190,11 @@
public synchronized void close() throws IOException {
if (socketChannel.isOpen()) {
engine.closeOutbound();
- new SslHandshake(this).handshake();
- socketChannel.close();
+ try {
+ new SslHandshake(this).handshake();
+ } finally {
+ socketChannel.close();
+ }
}
}
@@ -226,8 +233,11 @@
private void handleEndOfStreamQuietly() {
try {
- engine.closeInbound();
- close();
+ try {
+ engine.closeInbound();
+ } finally {
+ close();
+ }
} catch (Exception e) {
LOGGER.warn("failed to close socket gracefully", e);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/pom.xml b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/pom.xml
index 8fb63d6..5093c05 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/pom.xml
@@ -36,7 +36,7 @@
<dependency>
<groupId>org.apache.maven.plugin-tools</groupId>
<artifactId>maven-plugin-annotations</artifactId>
- <version>3.6.0</version>
+ <scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.maven</groupId>
@@ -45,7 +45,6 @@
<dependency>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-remote-resources-plugin</artifactId>
- <version>1.6.0</version>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
diff --git a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/ProjectFlag.java b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/ProjectFlag.java
index dd8fdf7..c7b2e52 100644
--- a/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/ProjectFlag.java
+++ b/hyracks-fullstack/hyracks/hyracks-maven-plugins/license-automation-plugin/src/main/java/org/apache/hyracks/maven/license/ProjectFlag.java
@@ -20,8 +20,8 @@
import static org.apache.hyracks.maven.license.LicenseUtil.toGav;
+import java.util.Arrays;
import java.util.Properties;
-import java.util.stream.Stream;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.tuple.Pair;
@@ -50,7 +50,7 @@
case IGNORE_MISSING_EMBEDDED_NOTICE:
case IGNORE_LICENSE_OVERRIDE:
case IGNORE_NOTICE_OVERRIDE:
- if (Stream.of(StringUtils.split(value, ",")).anyMatch(depObj.getVersion()::equals)) {
+ if (Arrays.asList(StringUtils.split(value, ",")).contains(depObj.getVersion())) {
licenseMojo.getProjectFlags().put(Pair.of(toGav(depObj), this), Boolean.TRUE);
} else {
licenseMojo.getLog().info(propName() + " defined on versions that *do not* match: " + value
@@ -59,6 +59,7 @@
break;
case ALTERNATE_LICENSE_FILE:
case ALTERNATE_NOTICE_FILE:
+ boolean found = false;
for (String spec : StringUtils.split(value, ",")) {
String[] specSplit = StringUtils.split(spec, ":");
if (specSplit.length != 2) {
@@ -66,8 +67,13 @@
}
if (specSplit[0].equals(depObj.getVersion())) {
licenseMojo.getProjectFlags().put(Pair.of(toGav(depObj), this), specSplit[1]);
+ found = true;
}
}
+ if (!found) {
+ licenseMojo.getLog().info(propName() + " defined on versions that *do not* match: " + value
+ + " for " + toGav(depObj));
+ }
break;
default:
throw new IllegalStateException("NYI: " + this);
diff --git a/hyracks-fullstack/hyracks/hyracks-net/pom.xml b/hyracks-fullstack/hyracks/hyracks-net/pom.xml
index 824098c..91d246d 100644
--- a/hyracks-fullstack/hyracks/hyracks-net/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-net/pom.xml
@@ -71,7 +71,7 @@
</dependency>
<dependency>
<groupId>org.mockito</groupId>
- <artifactId>mockito-all</artifactId>
+ <artifactId>mockito-core</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
diff --git a/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/AbstractChannelWriteInterface.java b/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/AbstractChannelWriteInterface.java
index 741ca8c..ee19de3 100644
--- a/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/AbstractChannelWriteInterface.java
+++ b/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/AbstractChannelWriteInterface.java
@@ -75,14 +75,14 @@
@GuardedBy("ChannelControlBlock")
private boolean computeWritability() {
- boolean writableDataPresent = currentWriteBuffer != null || !wiFullQueue.isEmpty();
+ if (!ecodeSent && ecode.get() == REMOTE_ERROR_CODE) {
+ return true;
+ }
+ boolean writableDataPresent = !ecodeSent && (currentWriteBuffer != null || !wiFullQueue.isEmpty());
if (writableDataPresent) {
return credits > 0;
}
- if (isPendingCloseWrite()) {
- return true;
- }
- return ecode.get() == REMOTE_ERROR_CODE && !ecodeSent;
+ return isPendingCloseWrite();
}
@Override
diff --git a/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/FullFrameChannelWriteInterface.java b/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/FullFrameChannelWriteInterface.java
index a7be3a6..e542a34 100644
--- a/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/FullFrameChannelWriteInterface.java
+++ b/hyracks-fullstack/hyracks/hyracks-net/src/main/java/org/apache/hyracks/net/protocols/muxdemux/FullFrameChannelWriteInterface.java
@@ -41,7 +41,15 @@
if (currentWriteBuffer == null) {
currentWriteBuffer = wiFullQueue.poll();
}
- if (currentWriteBuffer != null) {
+ if (!ecodeSent && ecode.get() == REMOTE_ERROR_CODE) {
+ writerState.getCommand().setChannelId(channelId);
+ writerState.getCommand().setCommandType(MuxDemuxCommand.CommandType.ERROR);
+ writerState.getCommand().setData(ecode.get());
+ writerState.reset(null, 0, null);
+ ecodeSent = true;
+ ccb.reportLocalEOS();
+ adjustChannelWritability();
+ } else if (currentWriteBuffer != null) {
int size = Math.min(currentWriteBuffer.remaining(), credits);
if (size > 0) {
credits -= size;
@@ -55,14 +63,6 @@
} else {
adjustChannelWritability();
}
- } else if (ecode.get() == REMOTE_ERROR_CODE && !ecodeSent) {
- writerState.getCommand().setChannelId(channelId);
- writerState.getCommand().setCommandType(MuxDemuxCommand.CommandType.ERROR);
- writerState.getCommand().setData(REMOTE_ERROR_CODE);
- writerState.reset(null, 0, null);
- ecodeSent = true;
- ccb.reportLocalEOS();
- adjustChannelWritability();
} else if (isPendingCloseWrite()) {
writerState.getCommand().setChannelId(channelId);
writerState.getCommand().setCommandType(MuxDemuxCommand.CommandType.CLOSE_CHANNEL);
diff --git a/hyracks-fullstack/hyracks/hyracks-net/src/test/java/org/apache/hyracks/net/tests/FullFrameChannelReadInterfaceTest.java b/hyracks-fullstack/hyracks/hyracks-net/src/test/java/org/apache/hyracks/net/tests/FullFrameChannelReadInterfaceTest.java
index 2180d1d..cf0ae08 100644
--- a/hyracks-fullstack/hyracks/hyracks-net/src/test/java/org/apache/hyracks/net/tests/FullFrameChannelReadInterfaceTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-net/src/test/java/org/apache/hyracks/net/tests/FullFrameChannelReadInterfaceTest.java
@@ -114,7 +114,7 @@
final ChannelControlBlock ccb = Mockito.mock(ChannelControlBlock.class);
Mockito.when(ccb.isRemotelyClosed()).thenReturn(false);
Mockito.doAnswer(invocation -> {
- final Integer delta = invocation.getArgumentAt(0, Integer.class);
+ final Integer delta = invocation.getArgument(0);
credit.addAndGet(delta);
synchronized (credit) {
credit.notifyAll();
@@ -128,7 +128,7 @@
final ISocketChannel sc = Mockito.mock(ISocketChannel.class);
Mockito.when(sc.read(Mockito.any(ByteBuffer.class))).thenAnswer(invocation -> {
ccb.addPendingCredits(-FRAME_SIZE);
- final ByteBuffer buffer = invocation.getArgumentAt(0, ByteBuffer.class);
+ final ByteBuffer buffer = invocation.getArgument(0);
while (buffer.hasRemaining()) {
buffer.put((byte) 0);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/pom.xml
index fa74184..0ccdaaa 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/pom.xml
@@ -67,25 +67,22 @@
</dependency>
<dependency>
<groupId>org.mockito</groupId>
- <artifactId>mockito-all</artifactId>
+ <artifactId>mockito-core</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.powermock</groupId>
- <artifactId>powermock-api-mockito</artifactId>
- <version>1.6.2</version>
+ <artifactId>powermock-api-mockito2</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.powermock</groupId>
<artifactId>powermock-module-junit4</artifactId>
- <version>1.6.2</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.powermock</groupId>
<artifactId>powermock-core</artifactId>
- <version>1.6.2</version>
<scope>test</scope>
</dependency>
<dependency>
@@ -107,5 +104,18 @@
<groupId>org.apache.commons</groupId>
<artifactId>commons-lang3</artifactId>
</dependency>
+ <dependency>
+ <groupId>com.fasterxml.jackson.core</groupId>
+ <artifactId>jackson-databind</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.logging.log4j</groupId>
+ <artifactId>log4j-api</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hyracks</groupId>
+ <artifactId>hyracks-util</artifactId>
+ <version>${project.version}</version>
+ </dependency>
</dependencies>
</project>
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
index 3a062af..f62f2d1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
@@ -69,11 +69,16 @@
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
import org.apache.hyracks.storage.common.buffercache.IPageWriteCallback;
import org.apache.hyracks.storage.common.file.BufferedFileHandle;
+import org.apache.hyracks.util.JSONUtil;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import com.fasterxml.jackson.databind.node.ObjectNode;
public class BTree extends AbstractTreeIndex {
public static final float DEFAULT_FILL_FACTOR = 0.7f;
-
+ private static final Logger LOGGER = LogManager.getLogger();
private static final long RESTART_OP = Long.MIN_VALUE;
private static final long FULL_RESTART_OP = Long.MIN_VALUE + 1;
private static final int MAX_RESTARTS = 10;
@@ -1086,6 +1091,7 @@
}
((IBTreeLeafFrame) leafFrame).insertSorted(tuple);
} catch (HyracksDataException | RuntimeException e) {
+ logState(tuple, e);
handleException();
throw e;
}
@@ -1194,6 +1200,24 @@
public void abort() throws HyracksDataException {
super.handleException();
}
+
+ private void logState(ITupleReference tuple, Exception e) {
+ try {
+ ObjectNode state = JSONUtil.createObject();
+ state.set("leafFrame", leafFrame.getState());
+ state.set("interiorFrame", interiorFrame.getState());
+ int tupleSize = Math.max(leafFrame.getBytesRequiredToWriteTuple(tuple),
+ interiorFrame.getBytesRequiredToWriteTuple(tuple));
+ state.put("tupleSize", tupleSize);
+ state.put("spaceNeeded", tupleWriter.bytesRequired(tuple) + slotSize);
+ state.put("spaceUsed", leafFrame.getBuffer().capacity() - leafFrame.getTotalFreeSpace());
+ state.put("leafMaxBytes", leafMaxBytes);
+ state.put("maxTupleSize", maxTupleSize);
+ LOGGER.error("failed to add tuple {}", state, e);
+ } catch (Throwable t) {
+ e.addSuppressed(t);
+ }
+ }
}
@SuppressWarnings("rawtypes")
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
index 44af086..0600add 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
@@ -73,7 +73,7 @@
import org.powermock.modules.junit4.PowerMockRunner;
@RunWith(PowerMockRunner.class)
-@PowerMockIgnore("javax.management.*")
+@PowerMockIgnore({ "javax.management.*", "javax.xml.*", "com.sun.org.apache.xerces.*", "org.w3c.*", "org.xml.*" })
@PrepareForTest({ BTreeUtils.class, FrameTupleAccessor.class, ArrayTupleBuilder.class,
IndexSearchOperatorNodePushable.class, FrameUtils.class, FrameTupleAppender.class })
public class FramewriterTest {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml b/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml
index 31463b7..f8cb226 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/pom.xml
@@ -102,12 +102,16 @@
</dependency>
<dependency>
<groupId>org.mockito</groupId>
- <artifactId>mockito-all</artifactId>
+ <artifactId>mockito-core</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
</dependency>
+ <dependency>
+ <groupId>com.fasterxml.jackson.core</groupId>
+ <artifactId>jackson-core</artifactId>
+ </dependency>
</dependencies>
</project>
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITreeIndexFrame.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITreeIndexFrame.java
index dc59612..18d5653 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITreeIndexFrame.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ITreeIndexFrame.java
@@ -28,6 +28,9 @@
import org.apache.hyracks.storage.common.buffercache.IBufferCache;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
import org.apache.hyracks.storage.common.buffercache.IExtraPageBlockHelper;
+import org.apache.hyracks.util.JSONUtil;
+
+import com.fasterxml.jackson.databind.node.ObjectNode;
public interface ITreeIndexFrame {
@@ -122,4 +125,14 @@
public ITupleReference getLeftmostTuple() throws HyracksDataException;
public ITupleReference getRightmostTuple() throws HyracksDataException;
+
+ default ObjectNode getState() {
+ ObjectNode json = JSONUtil.createObject();
+ json.put("tupleCount", getTupleCount());
+ json.put("freeSpaceOff", getFreeSpaceOff());
+ json.put("level", getLevel());
+ json.put("pageLsn", getPageLsn());
+ json.put("totalFreeSpace", getTotalFreeSpace());
+ return json;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/build/IndexBuilder.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/build/IndexBuilder.java
index f62860a..45bfed1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/build/IndexBuilder.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/build/IndexBuilder.java
@@ -47,6 +47,7 @@
protected final IResourceFactory localResourceFactory;
protected final boolean durable;
private final IResourceIdFactory resourceIdFactory;
+ private final String resourceRelPath;
/*
* Ideally, we should not pass resource id factory to the constructor since we can obtain it through
@@ -62,6 +63,7 @@
this.localResourceFactory = localResourceFactory;
this.durable = durable;
this.resourceRef = resourceRef;
+ resourceRelPath = resourceRef.getRelativePath();
}
@Override
@@ -72,35 +74,34 @@
// physical artifact that the LocalResourceRepository is managing (e.g. a file containing the resource Id).
// Once the index has been created, a new resource Id can be generated.
ILocalResourceRepository localResourceRepository = storageManager.getLocalResourceRepository(ctx);
- LocalResource lr = localResourceRepository.get(resourceRef.getRelativePath());
+ LocalResource lr = localResourceRepository.get(resourceRelPath);
long resourceId = lr == null ? -1 : lr.getId();
if (resourceId != -1) {
- localResourceRepository.delete(resourceRef.getRelativePath());
+ localResourceRepository.delete(resourceRelPath);
}
resourceId = resourceIdFactory.createId();
IResource resource = localResourceFactory.createResource(resourceRef);
lr = new LocalResource(resourceId, ITreeIndexFrame.Constants.VERSION, durable, resource);
- IIndex index = lcManager.get(resourceRef.getRelativePath());
+ IIndex index = lcManager.get(resourceRelPath);
if (index != null) {
//how is this right?????????? <needs to be fixed>
//The reason for this is to handle many cases such as:
//1. Crash while delete index is running (we don't do global cleanup on restart)
//2. Node leaves and then join with old data
- LOGGER.log(Level.WARN,
- "Removing existing index on index create for the index: " + resourceRef.getRelativePath());
- lcManager.unregister(resourceRef.getRelativePath());
+ LOGGER.log(Level.WARN, "Removing existing index on index create for the index: " + resourceRelPath);
+ lcManager.unregister(resourceRelPath);
index.destroy();
} else {
- if (resourceRef.getFile().exists()) {
+ final FileReference resolvedResourceRef = ctx.getIoManager().resolve(resourceRelPath);
+ if (resolvedResourceRef.getFile().exists()) {
// Index is not registered but the index file exists
// This is another big problem that we need to disallow soon
// We can only disallow this if we have a global cleanup after crash
// on reboot
- LOGGER.log(Level.WARN,
- "Deleting " + resourceRef.getRelativePath()
- + " on index create. The index is not registered"
- + " but the file exists in the filesystem");
- IoUtil.delete(resourceRef);
+ LOGGER.warn(
+ "Deleting {} on index create. The index is not registered but the file exists in the filesystem",
+ resolvedResourceRef);
+ IoUtil.delete(resolvedResourceRef);
}
index = resource.createInstance(ctx);
}
@@ -110,7 +111,7 @@
} catch (IOException e) {
throw HyracksDataException.create(e);
}
- lcManager.register(resourceRef.getRelativePath(), index);
+ lcManager.register(resourceRelPath, index);
}
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDropOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDropOperatorNodePushable.java
index 8490c6a..e48db2b 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDropOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexDropOperatorNodePushable.java
@@ -96,11 +96,11 @@
}
private boolean isIgnorable(HyracksDataException e) {
- return e.getErrorCode() == INDEX_DOES_NOT_EXIST && options.contains(IF_EXISTS);
+ return e.matches(INDEX_DOES_NOT_EXIST) && options.contains(IF_EXISTS);
}
private boolean canRetry(HyracksDataException e) throws HyracksDataException {
- if (e.getErrorCode() == CANNOT_DROP_IN_USE_INDEX && options.contains(WAIT_ON_IN_USE)) {
+ if (e.matches(CANNOT_DROP_IN_USE_INDEX) && options.contains(WAIT_ON_IN_USE)) {
if (maxWaitTimeMillis <= 0) {
return false;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java
index cf57d0e..9ff9249 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java
@@ -114,7 +114,7 @@
indexAccessor.insert(tuple);
} catch (HyracksDataException e) {
// ignore that exception to allow inserting existing keys which becomes an NoOp
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
throw e;
}
}
@@ -132,8 +132,8 @@
try {
indexAccessor.delete(tuple);
} catch (HyracksDataException e) {
- // ingnore that exception to allow deletions of non-existing keys
- if (e.getErrorCode() != ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
+ // ignore that exception to allow deletions of non-existing keys
+ if (!e.matches(ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)) {
throw e;
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/frames/TreeIndexNSMFrame.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/frames/TreeIndexNSMFrame.java
index 6106358..08d4564 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/frames/TreeIndexNSMFrame.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/frames/TreeIndexNSMFrame.java
@@ -31,6 +31,10 @@
import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
import org.apache.hyracks.storage.am.common.ophelpers.SlotOffTupleOff;
import org.apache.hyracks.storage.common.buffercache.ICachedPage;
+import org.apache.hyracks.util.JSONUtil;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.node.ObjectNode;
public abstract class TreeIndexNSMFrame implements ITreeIndexFrame {
@@ -354,4 +358,20 @@
return frameTuple;
}
}
+
+ @Override
+ public ObjectNode getState() {
+ ObjectNode state = ITreeIndexFrame.super.getState();
+ state.put("largeFlag", getLargeFlag());
+ return state;
+ }
+
+ @Override
+ public String toString() {
+ try {
+ return JSONUtil.convertNode(getState());
+ } catch (JsonProcessingException e) {
+ return "failed to convert json";
+ }
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IndexWithBuddyBulkLoader.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IndexWithBuddyBulkLoader.java
index 17dfb1d..494dd52 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IndexWithBuddyBulkLoader.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IndexWithBuddyBulkLoader.java
@@ -40,7 +40,7 @@
buddyBTreeBulkLoader.add(tuple);
} catch (HyracksDataException e) {
//deleting a key multiple times is OK
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
cleanupArtifacts();
throw e;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
index 068df9a..f8ad664 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java
@@ -175,7 +175,7 @@
try {
ctx.getCurrentDeletedKeysBTreeAccessors().insert(ctx.getKeysOnlyTuple());
} catch (HyracksDataException e) {
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
// Key has already been deleted.
LOGGER.log(Level.WARN, "Failure during index delete operation", e);
throw e;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
index a7ce35c..9c166b0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/org/apache/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndex.java
@@ -119,7 +119,7 @@
try {
btreeAccessor.insert(insertTuple);
} catch (HyracksDataException e) {
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
// This exception may be caused by duplicate tokens in the same insert "document".
// We ignore such duplicate tokens in all inverted-index implementations, hence
// we can safely ignore this exception.
@@ -139,7 +139,7 @@
try {
btreeAccessor.delete(deleteTuple);
} catch (HyracksDataException e) {
- if (e.getErrorCode() != ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
+ if (!e.matches(ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)) {
// Ignore this exception, since a document may have duplicate tokens.
throw e;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
index b04570e..1b847a0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/AbstractLSMRTree.java
@@ -220,7 +220,7 @@
try {
ctx.getCurrentMutableBTreeAccessor().insert(indexTuple);
} catch (HyracksDataException e) {
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
// Do nothing, because one delete tuple is enough to indicate
// that all the corresponding insert tuples are deleted
throw e;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
index 92449e4..7afecd8 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/org/apache/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java
@@ -417,7 +417,7 @@
} catch (HyracksDataException e) {
// Do nothing, because one delete tuple is enough to indicate
// that all the corresponding insert tuples are deleted
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
throw e;
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
index 5afddef..f1fe86f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/buffercache/BufferCache.java
@@ -527,8 +527,8 @@
read(cPage);
return;
} catch (HyracksDataException readException) {
- if (readException.getErrorCode() == ErrorCode.CANNOT_READ_CLOSED_FILE && i <= MAX_PAGE_READ_ATTEMPTS) {
- /**
+ if (readException.matches(ErrorCode.CANNOT_READ_CLOSED_FILE) && i != MAX_PAGE_READ_ATTEMPTS) {
+ /*
* if the read failure was due to another thread closing the file channel because
* it was interrupted, we will try to read again since the interrupted thread
* will re-open the file.
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/pom.xml b/hyracks-fullstack/hyracks/hyracks-test-support/pom.xml
index 7d2d2ae..4f8f4f4 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/pom.xml
@@ -116,5 +116,9 @@
<groupId>commons-io</groupId>
<artifactId>commons-io</artifactId>
</dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-core</artifactId>
+ </dependency>
</dependencies>
</project>
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexExamplesTest.java
index c37d781..3560230 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexExamplesTest.java
@@ -118,7 +118,7 @@
try {
indexAccessor.insert(tuple);
} catch (HyracksDataException e) {
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
throw e;
}
}
@@ -285,7 +285,7 @@
try {
indexAccessor.insert(tuple);
} catch (HyracksDataException e) {
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
throw e;
}
}
@@ -373,7 +373,7 @@
try {
indexAccessor.insert(tuple);
} catch (HyracksDataException e) {
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
throw e;
}
}
@@ -470,7 +470,7 @@
indexAccessor.insert(tuple);
insDone++;
} catch (HyracksDataException e) {
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
throw e;
}
}
@@ -492,7 +492,7 @@
indexAccessor.delete(tuple);
delDone++;
} catch (HyracksDataException e) {
- if (e.getErrorCode() != ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
+ if (!e.matches(ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)) {
throw e;
}
}
@@ -574,7 +574,7 @@
try {
indexAccessor.insert(tuple);
} catch (HyracksDataException e) {
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
throw e;
}
}
@@ -742,8 +742,8 @@
try {
bulkLoader.add(tuple);
} catch (HyracksDataException e) {
- if (e.getErrorCode() == ErrorCode.UNSORTED_LOAD_INPUT || e.getErrorCode() == ErrorCode.DUPLICATE_KEY
- || e.getErrorCode() == ErrorCode.DUPLICATE_LOAD_INPUT) {
+ if (e.matchesAny(ErrorCode.UNSORTED_LOAD_INPUT, ErrorCode.DUPLICATE_KEY,
+ ErrorCode.DUPLICATE_LOAD_INPUT)) {
if (j != i) {
fail("Unexpected exception: " + e.getMessage());
}
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexTestUtils.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexTestUtils.java
index d3dac3e..c974987 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexTestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/btree/OrderedIndexTestUtils.java
@@ -309,7 +309,7 @@
}
} catch (HyracksDataException e) {
// Ignore duplicate key insertions.
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
throw e;
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/TreeIndexTestUtils.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/TreeIndexTestUtils.java
index 0c7b5f9..a298fe5 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/TreeIndexTestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/common/TreeIndexTestUtils.java
@@ -237,7 +237,7 @@
} catch (HyracksDataException e) {
// We set expected values only after insertion succeeds because
// we ignore duplicate keys.
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
throw e;
}
}
@@ -291,7 +291,7 @@
} catch (HyracksDataException e) {
// We set expected values only after insertion succeeds because
// we ignore duplicate keys.
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
throw e;
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
index 48ae036..1fb5905 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java
@@ -169,7 +169,7 @@
try {
indexAccessor.insert(tuple);
} catch (HyracksDataException e) {
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
throw e;
}
}
@@ -574,7 +574,7 @@
try {
indexAccessor.insert(tuple);
} catch (HyracksDataException e) {
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
throw e;
}
}
@@ -698,7 +698,7 @@
try {
indexAccessor.insert(tuple);
} catch (HyracksDataException e) {
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
throw e;
}
}
@@ -715,7 +715,7 @@
indexAccessor.delete(tuple);
delDone++;
} catch (HyracksDataException e) {
- if (e.getErrorCode() != ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
+ if (!e.matches(ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)) {
throw e;
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/RTreeTestUtils.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/RTreeTestUtils.java
index 92f5055..55dcb5b 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/RTreeTestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/storage/am/rtree/RTreeTestUtils.java
@@ -124,7 +124,7 @@
} catch (HyracksDataException e) {
// We set expected values only after insertion succeeds because
// we ignore duplicate keys.
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
throw e;
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/FormattedExceptionTestBase.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/FormattedExceptionTestBase.java
new file mode 100644
index 0000000..c912097
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/FormattedExceptionTestBase.java
@@ -0,0 +1,299 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.test.support;
+
+import java.io.File;
+import java.io.Serializable;
+import java.lang.reflect.Array;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Executable;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IError;
+import org.apache.hyracks.api.exceptions.IFormattedException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.mockito.Mockito;
+
+@RunWith(Parameterized.class)
+public class FormattedExceptionTestBase {
+ private static final Logger LOGGER = LogManager.getLogger();
+ private static final Serializable[] FAKE_ARGS = new Serializable[] { "one", 2, 3.0f, 4.0d, (byte) 0x05 };
+ private static final Serializable[] EMPTY_ARGS = new Serializable[0];
+ private static final Class SERIALIZABLE_ARR_CLASS = EMPTY_ARGS.getClass();
+ private static final org.apache.hyracks.api.exceptions.ErrorCode HYR_ERROR_CODE =
+ random(org.apache.hyracks.api.exceptions.ErrorCode.values());
+ private static final StackTraceElement[] STACK_TRACE = new Throwable().getStackTrace();
+ private static final SourceLocation SOURCE_LOCATION = new SourceLocation(99, 9);
+ private static final ClassLoader CLASSLOADER = FormattedExceptionTestBase.class.getClassLoader();
+ private static final Field THROWABLE_DETAIL_MESSAGE = getDeclaredAccessibleField(Throwable.class, "detailMessage");
+ private static final Field THROWABLE_CAUSE = getDeclaredAccessibleField(Throwable.class, "cause");
+ private static final UnsupportedOperationException FAKE_THROWABLE = new UnsupportedOperationException();
+ private static final int PUBLIC_STATIC = (Modifier.PUBLIC | Modifier.STATIC);
+ private static Collection<Class<? extends IFormattedException>> exceptionClasses;
+ protected static Set<Class<? extends IFormattedException>> roots;
+ private static Set<Executable> publicContractOverrides = new HashSet<>();
+ protected final Executable action;
+ protected final Class<? extends IFormattedException> root;
+ private static final Map<Pair<Class<? extends IFormattedException>, Object>, Field> rootFields = new HashMap<>();
+ private static final Set<Class> visited = new HashSet<>();
+ protected static Predicate<String> classSelector = className -> true;
+
+ protected static Iterable<Object[]> defineParameters() throws ClassNotFoundException {
+ initClasses();
+ List<Object[]> tests = new ArrayList<>();
+ for (Class<? extends IFormattedException> clazz : exceptionClasses) {
+ Class<? extends IFormattedException> root = roots.stream().filter(c -> c.isAssignableFrom(clazz)).findAny()
+ .orElseThrow(IllegalStateException::new);
+ final Constructor<?>[] declaredConstructors = clazz.getDeclaredConstructors();
+ for (Constructor<?> ctor : declaredConstructors) {
+ tests.add(new Object[] { clazz.getSimpleName() + ".<init>"
+ + Stream.of(ctor.getParameterTypes()).map(Class::getSimpleName).collect(Collectors.toList()),
+ ctor, root });
+ }
+ int methods = 0;
+ for (Method m : clazz.getDeclaredMethods()) {
+ if ((m.getModifiers() & PUBLIC_STATIC) == PUBLIC_STATIC) {
+ methods++;
+ tests.add(new Object[] { clazz.getSimpleName() + "." + m.getName()
+ + Stream.of(m.getParameterTypes()).map(Class::getSimpleName).collect(Collectors.toList()),
+ m, root });
+ }
+ }
+ LOGGER.info("discovered {} ctors, {} methods for class {}", declaredConstructors.length, methods, clazz);
+ }
+ return tests;
+ }
+
+ protected static void addPublicContractOverride(Executable override) {
+ publicContractOverrides.add(override);
+ }
+
+ public FormattedExceptionTestBase(String desc, Executable action, Class<? extends IFormattedException> root) {
+ this.action = action;
+ this.root = root;
+ }
+
+ @Test
+ public void test() throws Exception {
+ if (Modifier.isPublic(action.getModifiers())) {
+ try {
+ checkPublicContract();
+ } catch (AssertionError e) {
+ if (publicContractOverrides.contains(action)) {
+ LOGGER.info("ignoring public contract vioilation for override executable: " + action);
+ } else {
+ throw e;
+ }
+ }
+ }
+ if (action.getName().equals("create") || action instanceof Constructor) {
+ checkParameterPropagation(action);
+ }
+ }
+
+ protected void checkPublicContract() {
+ for (Class type : action.getParameterTypes()) {
+ Assert.assertNotEquals("generic IError forbidden on public ctor or static method", type, IError.class);
+ }
+ }
+
+ private void checkParameterPropagation(Executable factory) throws Exception {
+ Object[] args = Stream.of(factory.getParameterTypes()).map(this::defaultValue).toArray(Object[]::new);
+ factory.setAccessible(true);
+ Field paramsField = rootParamsField();
+ Object instance = factory instanceof Constructor ? ((Constructor) factory).newInstance(args)
+ : ((Method) factory).invoke(null, args);
+ Serializable[] params = (Serializable[]) paramsField.get(instance);
+ IError error = null;
+ for (Class type : factory.getParameterTypes()) {
+ if (type.equals(paramsField.getType())) {
+ Assert.assertArrayEquals(FAKE_ARGS, params);
+ } else if (SourceLocation.class.isAssignableFrom(type)) {
+ final Object value = rootSrcLocField().get(instance);
+ Assert.assertEquals("source location is wrong, was: " + value, SOURCE_LOCATION, value);
+ } else if (IError.class.isAssignableFrom(type)) {
+ error = (IError) rootErrorField().get(instance);
+ Assert.assertNotNull("error object", error);
+ } else if (type.equals(Throwable.class)) {
+ Assert.assertEquals(FAKE_THROWABLE, THROWABLE_CAUSE.get(instance));
+ }
+ }
+ if (error != null) {
+ Assert.assertEquals(error.component(), getRootField("component").get(instance));
+ Assert.assertEquals(error.intValue(), getRootField("errorCode").get(instance));
+ Assert.assertEquals(error.errorMessage(), THROWABLE_DETAIL_MESSAGE.get(instance));
+ }
+ }
+
+ protected Field rootParamsField() {
+ return getRootField(SERIALIZABLE_ARR_CLASS);
+ }
+
+ protected Field rootErrorField() {
+ return getRootField(IError.class);
+ }
+
+ protected Field rootSrcLocField() {
+ return getRootField(SourceLocation.class);
+ }
+
+ protected Field getRootField(Class<?> type) {
+ return rootFields.computeIfAbsent(Pair.of(root, type),
+ key -> Stream.of(root.getDeclaredFields()).filter(f -> f.getType().equals(type))
+ .peek(f -> f.setAccessible(true)).findAny().orElseThrow(IllegalStateException::new));
+ }
+
+ protected Field getRootField(String name) {
+ return rootFields.computeIfAbsent(Pair.of(root, name), key -> getDeclaredAccessibleField(root, name));
+ }
+
+ protected static Field getDeclaredAccessibleField(Class clazz, String name) {
+ try {
+ Field field = clazz.getDeclaredField(name);
+ field.setAccessible(true);
+ return field;
+ } catch (NoSuchFieldException e) {
+ throw new IllegalStateException(e);
+ }
+ }
+
+ protected Object defaultValue(Class type) {
+ switch (type.getName()) {
+ case "int":
+ return 0;
+ case "float":
+ return 0.0f;
+ case "double":
+ return 0.0d;
+ case "long":
+ return 0L;
+ case "boolean":
+ return false;
+ case "short":
+ return (short) 0;
+ case "byte":
+ return (byte) 0;
+ case "[Ljava.io.Serializable;":
+ return FAKE_ARGS;
+ case "java.lang.Throwable":
+ return FAKE_THROWABLE;
+ case "org.apache.hyracks.api.exceptions.IError":
+ case "org.apache.hyracks.api.exceptions.ErrorCode":
+ return HYR_ERROR_CODE;
+ case "[Ljava.lang.StackTraceElement;":
+ return STACK_TRACE;
+ case "org.apache.hyracks.api.exceptions.SourceLocation":
+ return SOURCE_LOCATION;
+ case "org.apache.hyracks.api.exceptions.HyracksDataException":
+ HyracksDataException hde = Mockito.mock(HyracksDataException.class);
+ Mockito.when(hde.getError()).thenReturn(Optional.empty());
+ return hde;
+ default:
+ if (type.isArray()) {
+ return Array.newInstance(type.getComponentType(), 0);
+ } else if (type.isEnum()) {
+ return random(type.getEnumConstants());
+ } else if (type.isAnonymousClass() || Modifier.isFinal(type.getModifiers())) {
+ if (visited.add(type)) {
+ LOGGER.info("defaulting to null for un-mockable class {}", type.getName());
+ }
+ return null;
+ }
+ if (visited.add(type)) {
+ LOGGER.info("defaulting to mock for unmapped class {}", type.getName());
+ }
+ return Mockito.mock(type);
+ }
+ }
+
+ protected static <T> T random(T[] values) {
+ return values[RandomUtils.nextInt(0, values.length)];
+ }
+
+ @SuppressWarnings("unchecked")
+ private static void initClasses() throws ClassNotFoundException {
+ LOGGER.info("discovering instances of IFormattedException");
+ //noinspection unchecked
+ final Class<IFormattedException> clazz =
+ (Class) Class.forName(IFormattedException.class.getName(), false, CLASSLOADER);
+ exceptionClasses =
+ getInstanceClasses(clazz).sorted(Comparator.comparing(Class::getName)).collect(Collectors.toList());
+ exceptionClasses.remove(clazz);
+ LOGGER.info("found {} instances of IFormattedException: {}", exceptionClasses.size(), exceptionClasses);
+
+ roots = exceptionClasses.stream().map(ex -> {
+ while (IFormattedException.class.isAssignableFrom(ex.getSuperclass())) {
+ ex = (Class<? extends IFormattedException>) ex.getSuperclass();
+ }
+ return ex;
+ }).collect(Collectors.toSet());
+ LOGGER.info("found {} roots: {}", roots.size(), roots);
+ exceptionClasses.removeAll(roots);
+ }
+
+ @SuppressWarnings("unchecked")
+ private static <T> Stream<Class<? extends T>> getInstanceClasses(Class<T> clazz) {
+ return (Stream) getProductClasses().filter(name -> name.matches(".*(Exception|Error|Warning).*")).map(name -> {
+ try {
+ return Class.forName(name, false, CLASSLOADER);
+ } catch (Throwable e) {
+ LOGGER.warn("unable to open {} due to: {}", name, String.valueOf(e));
+ return null;
+ }
+ }).filter(Objects::nonNull).filter(clazz::isAssignableFrom);
+ }
+
+ private static Stream<String> getProductClasses() {
+ String[] cp = System.getProperty("java.class.path").split(File.pathSeparator);
+ return Stream.of(cp).map(File::new).filter(File::isDirectory)
+ .flatMap(FormattedExceptionTestBase::extractClassFiles).map(name -> name.replace("/", "."))
+ .filter(classSelector).map(name -> name.replaceAll("\\.class$", "")).sorted();
+ }
+
+ private static Stream<? extends String> extractClassFiles(File dir) {
+ final int beginIndex = dir.toString().length() + 1;
+ return FileUtils.listFiles(dir, new String[] { "class" }, true).stream()
+ .map(file -> file.getAbsolutePath().substring(beginIndex));
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestUtils.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestUtils.java
index e44480c..3f78234 100644
--- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestUtils.java
@@ -51,6 +51,7 @@
public class TestUtils {
+ private static final int DEFAULT_FRAME_SIZE = 32768;
public static final IWarningCollector NOOP_WARNING_COLLECTOR = new IWarningCollector() {
@Override
public void warn(Warning warning) {
@@ -68,6 +69,10 @@
}
};
+ public static IHyracksTaskContext createHyracksTask() {
+ return create(DEFAULT_FRAME_SIZE);
+ }
+
public static IHyracksTaskContext create(int frameSize) {
IOManager ioManager = null;
try {
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeStatsTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeStatsTest.java
index dfaa7f9..7a24727 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeStatsTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeStatsTest.java
@@ -136,7 +136,7 @@
try {
indexAccessor.insert(tuple);
} catch (HyracksDataException e) {
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
e.printStackTrace();
throw e;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeUpdateSearchTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeUpdateSearchTest.java
index 8e5c77d..5b86b4f 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeUpdateSearchTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/BTreeUpdateSearchTest.java
@@ -117,7 +117,7 @@
try {
indexAccessor.insert(insertTuple);
} catch (HyracksDataException hde) {
- if (hde.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!hde.matches(ErrorCode.DUPLICATE_KEY)) {
hde.printStackTrace();
throw hde;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/multithread/BTreeTestWorker.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/multithread/BTreeTestWorker.java
index a67450b..5d74fc9 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/multithread/BTreeTestWorker.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/org/apache/hyracks/storage/am/btree/multithread/BTreeTestWorker.java
@@ -62,7 +62,7 @@
try {
accessor.insert(tuple);
} catch (HyracksDataException e) {
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
// Ignore duplicate keys, since we get random tuples.
throw e;
}
@@ -79,7 +79,7 @@
try {
accessor.delete(deleteTuple);
} catch (HyracksDataException e) {
- if (e.getErrorCode() != ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
+ if (!e.matches(ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)) {
// Ignore non-existant keys, since we get random tuples.
throw e;
}
@@ -91,8 +91,8 @@
accessor.update(tuple);
} catch (HyracksDataException e) {
// Ignore non-existant keys, since we get random tuples.
- if (e.getErrorCode() != ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY
- && e.getErrorCode() != ErrorCode.INDEX_NOT_UPDATABLE) {
+ if (!e.matches(ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)
+ && !e.matches(ErrorCode.INDEX_NOT_UPDATABLE)) {
// Ignore non-existant keys, since we get random tuples.
// Ignore not updateable exception due to numKeys == numFields.
throw e;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeScanDiskComponentsTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeScanDiskComponentsTest.java
index b47f5dd..c9c87bd 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeScanDiskComponentsTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeScanDiskComponentsTest.java
@@ -205,7 +205,7 @@
try {
ctx.getIndexAccessor().upsert(tuple);
} catch (HyracksDataException e) {
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
throw e;
}
}
@@ -222,7 +222,7 @@
try {
ctx.getIndexAccessor().delete(deleteTuple);
} catch (HyracksDataException e) {
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
throw e;
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java
index bbe0966..8d82405 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeUpdateInPlaceScanDiskComponentsTest.java
@@ -140,7 +140,7 @@
} catch (HyracksDataException e) {
// We set expected values only after insertion succeeds because
// we ignore duplicate keys.
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
throw e;
}
}
@@ -162,7 +162,7 @@
} catch (HyracksDataException e) {
// We set expected values only after insertion succeeds because
// we ignore duplicate keys.
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
throw e;
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/cursor/LSMBTreePointSearchCursorTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/cursor/LSMBTreePointSearchCursorTest.java
index 3c4ae51..5e5410c 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/cursor/LSMBTreePointSearchCursorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/cursor/LSMBTreePointSearchCursorTest.java
@@ -145,7 +145,7 @@
try {
indexAccessor.insert(tuple);
} catch (HyracksDataException e) {
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
e.printStackTrace();
throw e;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java
index 583fd5a..60a3fa7 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java
@@ -63,7 +63,7 @@
try {
accessor.insert(tuple);
} catch (HyracksDataException e) {
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
// Ignore duplicate keys, since we get random tuples.
throw e;
}
@@ -81,7 +81,7 @@
accessor.delete(deleteTuple);
} catch (HyracksDataException e) {
// Ignore non-existant keys, since we get random tuples.
- if (e.getErrorCode() != ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY) {
+ if (!e.matches(ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)) {
throw e;
}
}
@@ -91,8 +91,8 @@
try {
accessor.update(tuple);
} catch (HyracksDataException e) {
- if (e.getErrorCode() != ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY
- && e.getErrorCode() != ErrorCode.INDEX_NOT_UPDATABLE) {
+ if (!e.matches(ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)
+ && !e.matches(ErrorCode.INDEX_NOT_UPDATABLE)) {
// Ignore non-existant keys, since we get random tuples.
// Ignore not updateable exception due to numKeys == numFields.
throw e;
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/pom.xml b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/pom.xml
index c38ba3c..c9e4ba1 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/pom.xml
@@ -90,7 +90,7 @@
</dependency>
<dependency>
<groupId>org.mockito</groupId>
- <artifactId>mockito-all</artifactId>
+ <artifactId>mockito-core</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/ConcurrentMergePolicyTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/ConcurrentMergePolicyTest.java
index 098aa51..af0be25 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/ConcurrentMergePolicyTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/ConcurrentMergePolicyTest.java
@@ -166,7 +166,7 @@
Mockito.doAnswer(new Answer<Void>() {
@Override
public Void answer(InvocationOnMock invocation) throws Throwable {
- List<ILSMDiskComponent> mergedComponents = invocation.getArgumentAt(0, List.class);
+ List<ILSMDiskComponent> mergedComponents = invocation.getArgument(0);
mergedComponents.forEach(component -> {
mergedSizes.add(component.getComponentSize());
});
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/PrefixMergePolicyTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/PrefixMergePolicyTest.java
index 54fd3bb..2e7b4c6 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/PrefixMergePolicyTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/PrefixMergePolicyTest.java
@@ -196,7 +196,7 @@
Mockito.doAnswer(new Answer<Void>() {
@Override
public Void answer(InvocationOnMock invocation) throws Throwable {
- List<ILSMDiskComponent> mergedComponents = invocation.getArgumentAt(0, List.class);
+ List<ILSMDiskComponent> mergedComponents = invocation.getArgument(0);
if (mergedSizes != null) {
mergedComponents.forEach(component -> {
mergedSizes.add(component.getComponentSize());
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
index 2fcd96f..355b45c 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java
@@ -123,7 +123,7 @@
consumeCursorTuples(searchCursor);
} catch (HyracksDataException e) {
// Ignore.
- if (e.getErrorCode() != ErrorCode.OCCURRENCE_THRESHOLD_PANIC_EXCEPTION) {
+ if (!e.matches(ErrorCode.OCCURRENCE_THRESHOLD_PANIC_EXCEPTION)) {
throw e;
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
index 8dc9b07..d9177b1 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/org/apache/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java
@@ -591,7 +591,7 @@
accessor.search(resultCursor, searchPred);
} catch (HyracksDataException e) {
// ignore panic queries.
- if (e.getErrorCode() == ErrorCode.OCCURRENCE_THRESHOLD_PANIC_EXCEPTION) {
+ if (e.matches(ErrorCode.OCCURRENCE_THRESHOLD_PANIC_EXCEPTION)) {
panic = true;
} else {
throw e;
@@ -610,7 +610,7 @@
actualResults.add(Integer.valueOf(actual));
}
} catch (HyracksDataException e) {
- if (e.getErrorCode() == ErrorCode.OCCURRENCE_THRESHOLD_PANIC_EXCEPTION) {
+ if (e.matches(ErrorCode.OCCURRENCE_THRESHOLD_PANIC_EXCEPTION)) {
// Ignore panic queries.
continue;
} else {
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/AbstractLSMRTreeExamplesTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/AbstractLSMRTreeExamplesTest.java
index 39fb98f..9698578 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/AbstractLSMRTreeExamplesTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/org/apache/hyracks/storage/am/lsm/rtree/AbstractLSMRTreeExamplesTest.java
@@ -134,7 +134,7 @@
try {
indexAccessor.insert(tuple);
} catch (HyracksDataException e) {
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
throw e;
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
index faf0693..bef7ac7 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/org/apache/hyracks/storage/am/rtree/RTreeSearchCursorTest.java
@@ -168,7 +168,7 @@
try {
indexAccessor.insert(tuple);
} catch (HyracksDataException e) {
- if (e.getErrorCode() != ErrorCode.DUPLICATE_KEY) {
+ if (!e.matches(ErrorCode.DUPLICATE_KEY)) {
throw e;
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/pom.xml b/hyracks-fullstack/hyracks/hyracks-util/pom.xml
index e2da20a..621e6f6 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/pom.xml
+++ b/hyracks-fullstack/hyracks/hyracks-util/pom.xml
@@ -95,6 +95,10 @@
<groupId>org.apache.httpcomponents</groupId>
<artifactId>httpclient</artifactId>
</dependency>
+ <dependency>
+ <groupId>com.google.guava</groupId>
+ <artifactId>guava</artifactId>
+ </dependency>
</dependencies>
</project>
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java
index 4991f86..abd9fda 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ExitUtil.java
@@ -20,6 +20,9 @@
import java.lang.reflect.Field;
import java.util.IdentityHashMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
@@ -63,6 +66,8 @@
private static final ExitThread exitThread = new ExitThread();
private static final ShutdownWatchdog watchdogThread = new ShutdownWatchdog();
private static final MutableLong shutdownHaltDelay = new MutableLong(10 * 60 * 1000L); // 10 minutes default
+ private static final ExecutorService haltThreadDumpExecutor = Executors.newSingleThreadExecutor();
+ private static final long HALT_THREADDUMP_TIMEOUT_SECONDS = 60;
static {
watchdogThread.start();
@@ -97,11 +102,19 @@
}
public static synchronized void halt(int status, Level logLevel) {
- LOGGER.log(logLevel, "JVM halting with status {}; thread dump at halt: {}", status,
- ThreadDumpUtil.takeDumpString());
- // try to give time for the log to be emitted...
- LogManager.shutdown();
- Runtime.getRuntime().halt(status);
+ try {
+ Future<?> future = haltThreadDumpExecutor.submit(() -> {
+ LOGGER.log(logLevel, "JVM halting with status {}; thread dump at halt: {}", status,
+ ThreadDumpUtil.takeDumpString());
+ // try to give time for the log to be emitted...
+ LogManager.shutdown();
+ });
+ future.get(HALT_THREADDUMP_TIMEOUT_SECONDS, TimeUnit.SECONDS);
+ } catch (Exception e) {
+ LOGGER.warn("exception logging thread dump on halt", e);
+ } finally {
+ Runtime.getRuntime().halt(status);
+ }
}
public static boolean registerShutdownHook(Thread shutdownHook) {
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ILogRedactor.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ILogRedactor.java
index b133894..d36c77b 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ILogRedactor.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ILogRedactor.java
@@ -29,6 +29,14 @@
String userData(String text);
/**
+ * Redacts statement argument.
+ *
+ * @param text statement to redact.
+ * @return redacted statement.
+ */
+ String statement(String text);
+
+ /**
* Unredacts user data found in the argument.
*
* @param text text that contains some redacted user data.
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IRetryPolicy.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IRetryPolicy.java
index 29469d5..0d18a2b 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IRetryPolicy.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/IRetryPolicy.java
@@ -22,7 +22,7 @@
public interface IRetryPolicy {
/**
* @param failure
- * the cause of the failure
+ * the cause of the failure (this cannot be null)
* @return true if one more attempt should be done
*/
boolean retry(Throwable failure);
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/LogRedactionUtil.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/LogRedactionUtil.java
index 89c957e..fdce8c8 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/LogRedactionUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/LogRedactionUtil.java
@@ -28,6 +28,11 @@
}
@Override
+ public String statement(String text) {
+ return text;
+ }
+
+ @Override
public String unredactUserData(String text) {
return text;
}
@@ -45,6 +50,10 @@
return redactor.userData(text);
}
+ public static String statement(String text) {
+ return redactor.statement(text);
+ }
+
public static String unredactUserData(String text) {
return redactor.unredactUserData(text);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/MXHelper.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/MXHelper.java
index 247c001..c238b50 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/MXHelper.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/MXHelper.java
@@ -108,9 +108,8 @@
public static String getBootClassPath() {
try {
- return runtimeMXBean.getBootClassPath();
+ return runtimeMXBean.isBootClassPathSupported() ? runtimeMXBean.getBootClassPath() : null;
} catch (UnsupportedOperationException e) {
- // boot classpath is not supported in Java 9 and later
LOGGER.debug("ignoring exception calling RuntimeMXBean.getBootClassPath; returning null", e);
return null;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/NetworkUtil.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/NetworkUtil.java
index 3f6e90c..1780d9c 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/NetworkUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/NetworkUtil.java
@@ -32,6 +32,7 @@
import javax.net.ssl.SSLEngine;
+import org.apache.commons.lang3.tuple.Pair;
import org.apache.http.HttpHost;
import org.apache.http.client.utils.URIBuilder;
import org.apache.http.conn.util.InetAddressUtils;
@@ -103,6 +104,34 @@
return hosts;
}
+ public static String defaultPort(String maybeHostPort, int defaultPort) {
+ Pair<String, Integer> decoded = extractHostPort(maybeHostPort, defaultPort);
+ return encodeIPv6LiteralHost(decoded.getLeft()) + ":" + decoded.getRight();
+ }
+
+ public static Pair<String, Integer> extractHostPort(String maybeHostPort, int defaultPort) {
+ try {
+ int lastColon = maybeHostPort.lastIndexOf(':');
+ if (lastColon == -1) {
+ return Pair.of(maybeHostPort, defaultPort);
+ }
+ int closeBracket = maybeHostPort.lastIndexOf(']');
+ if (closeBracket == -1) {
+ if (InetAddressUtils.isIPv6Address(maybeHostPort)) {
+ return Pair.of(maybeHostPort, defaultPort);
+ }
+ return Pair.of(maybeHostPort.substring(0, lastColon),
+ Integer.parseInt(maybeHostPort.substring(lastColon + 1)));
+ } else if (closeBracket < lastColon) {
+ return Pair.of(decodeIPv6LiteralHost(maybeHostPort.substring(0, lastColon)),
+ Integer.parseInt(maybeHostPort.substring(lastColon + 1)));
+ }
+ return Pair.of(decodeIPv6LiteralHost(maybeHostPort), defaultPort);
+ } catch (Exception e) {
+ throw new IllegalArgumentException(e);
+ }
+ }
+
public static String encodeIPv6LiteralHost(String hostname) {
return InetAddressUtils.isIPv6Address(hostname) ? "[" + hostname + "]" : hostname;
}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/Span.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/Span.java
index 183cb6f..bc33e6d 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/Span.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/Span.java
@@ -21,18 +21,26 @@
import java.util.concurrent.TimeUnit;
public class Span {
- private final long startNanos;
private final long spanNanos;
+ private volatile long startNanos;
private Span(long span, TimeUnit unit) {
- startNanos = System.nanoTime();
spanNanos = unit.toNanos(span);
+ reset();
+ }
+
+ public void reset() {
+ startNanos = System.nanoTime();
}
public long getSpanNanos() {
return spanNanos;
}
+ public long getSpan(TimeUnit unit) {
+ return unit.convert(spanNanos, TimeUnit.NANOSECONDS);
+ }
+
public static Span start(long span, TimeUnit unit) {
return new Span(span, unit);
}
@@ -46,6 +54,15 @@
}
/**
+ * Sleep for the remainder of this span
+ *
+ * @throws InterruptedException
+ */
+ public void sleep() throws InterruptedException {
+ TimeUnit.NANOSECONDS.sleep(remaining(TimeUnit.NANOSECONDS));
+ }
+
+ /**
* Sleep for the minimum of the duration or the remaining of this span
*
* @param sleep
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/StorageUtil.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/StorageUtil.java
index e4969f0..0456dc7 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/StorageUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/StorageUtil.java
@@ -20,10 +20,13 @@
import java.util.HashMap;
import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
public class StorageUtil {
public static final int BASE = 1024;
+ private static final Pattern PATTERN = Pattern.compile("^(-?[.0-9]+)([A-Z]{0,2})$");
public enum StorageUnit {
BYTE("B", "b", 1),
@@ -96,43 +99,27 @@
* @throws IllegalArgumentException
*/
public static double getSizeInBytes(String s) {
- String sSpaceRemoved = s.replaceAll(" ", "");
- String sUpper = sSpaceRemoved.toUpperCase();
-
- // Default type
- StorageUtil.StorageUnit unitType;
-
- // If the length is 1, it should only contain a digit number.
- if (sUpper.length() == 1) {
- if (Character.isDigit(sUpper.charAt(0))) {
- unitType = StorageUnit.BYTE;
- } else {
- throw invalidFormatException(s);
- }
- } else if (sUpper.length() > 1) {
- String checkStr = sUpper.substring(sUpper.length() - 2);
- unitType = StorageUnit.lookupBySuffix(checkStr);
-
- if (unitType == null) {
- // The last suffix should be at least "B" or a digit to be qualified as byte unit string.
- char lastChar = sUpper.charAt(sUpper.length() - 1);
- if (sUpper.substring(sUpper.length() - 1).equals(StorageUnit.BYTE.toString())
- || Character.isDigit(lastChar)) {
- unitType = StorageUnit.BYTE;
- } else {
- throw invalidFormatException(s);
- }
- }
- } else {
- // String length is zero. We can't parse this string.
+ String valueAndUnit = s.replace(" ", "").toUpperCase();
+ Matcher matcher = PATTERN.matcher(valueAndUnit);
+ if (!matcher.find()) {
throw invalidFormatException(s);
}
- // Strip all unit suffixes such as KB, MB ...
- String sFinalVal = sUpper.replaceAll("[^-\\.0123456789]", "");
+ String value = matcher.group(1);
+ String unit = matcher.group(2);
- // Return the bytes.
- return unitType.toBytes(Double.parseDouble(sFinalVal));
+ // Default to bytes or find provided unit
+ StorageUnit unitType = !unit.isEmpty() ? StorageUnit.lookupBySuffix(unit) : StorageUnit.BYTE;
+ if (unitType == null) {
+ throw invalidFormatException(s);
+ }
+
+ try {
+ // Return the bytes.
+ return unitType.toBytes(Double.parseDouble(value));
+ } catch (NumberFormatException ex) {
+ throw invalidFormatException(s);
+ }
}
private static IllegalArgumentException invalidFormatException(String s) {
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThrowingConsumer.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThrowingConsumer.java
new file mode 100644
index 0000000..be9874a
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThrowingConsumer.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.util;
+
+import java.util.function.Consumer;
+
+import com.google.common.util.concurrent.UncheckedExecutionException;
+
+@FunctionalInterface
+public interface ThrowingConsumer<V> {
+ void process(V value) throws Exception;
+
+ @SuppressWarnings("Duplicates")
+ static <T> Consumer<T> asUnchecked(ThrowingConsumer<T> consumer) {
+ return input -> {
+ try {
+ consumer.process(input);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new UncheckedExecutionException(e);
+ } catch (Exception e) {
+ throw new UncheckedExecutionException(e);
+ }
+ };
+ }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThrowingFunction.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThrowingFunction.java
new file mode 100644
index 0000000..d18b41b
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThrowingFunction.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.util;
+
+@FunctionalInterface
+public interface ThrowingFunction<I, R> {
+ R process(I input) throws Exception;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThrowingIntConsumer.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThrowingIntConsumer.java
new file mode 100644
index 0000000..07575bb
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThrowingIntConsumer.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.util;
+
+import java.util.function.IntConsumer;
+
+import com.google.common.util.concurrent.UncheckedExecutionException;
+
+@FunctionalInterface
+public interface ThrowingIntConsumer {
+ void process(int value) throws Exception;
+
+ @SuppressWarnings("Duplicates")
+ static IntConsumer asUnchecked(ThrowingIntConsumer consumer) {
+ return input -> {
+ try {
+ consumer.process(input);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new UncheckedExecutionException(e);
+ } catch (Exception e) {
+ throw new UncheckedExecutionException(e);
+ }
+ };
+ }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThrowingSupplier.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThrowingSupplier.java
new file mode 100644
index 0000000..28f5e29
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThrowingSupplier.java
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.util;
+
+@FunctionalInterface
+public interface ThrowingSupplier<T> {
+ T get() throws Exception;
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/NetworkUtilTest.java b/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/NetworkUtilTest.java
new file mode 100644
index 0000000..c5d42c5
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/NetworkUtilTest.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.util;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class NetworkUtilTest {
+
+ private static final Logger LOGGER = LogManager.getLogger();
+
+ @Test
+ public void testDefaultPort() {
+ Assert.assertEquals("127.0.0.1:1234", NetworkUtil.defaultPort("127.0.0.1:1234", 9999));
+ Assert.assertEquals("127.0.0.1:9999", NetworkUtil.defaultPort("127.0.0.1", 9999));
+ Assert.assertEquals("[::1]:1234", NetworkUtil.defaultPort("[::1]:1234", 9999));
+ Assert.assertEquals("[::1]:9999", NetworkUtil.defaultPort("::1", 9999));
+ Assert.assertEquals("localhost.localdomain.local:9999",
+ NetworkUtil.defaultPort("localhost.localdomain.local", 9999));
+ Assert.assertEquals("localhost.localdomain.local:1234",
+ NetworkUtil.defaultPort("localhost.localdomain.local:1234", 9999));
+
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/StorageUnitTest.java b/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/StorageUnitTest.java
new file mode 100644
index 0000000..445d15f
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/test/java/org/apache/hyracks/util/StorageUnitTest.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.util;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class StorageUnitTest {
+
+ @Test
+ public void test() {
+ // Valid cases
+ double result1NoUnit = StorageUtil.getSizeInBytes("1"); // Defaults to bytes
+ Assert.assertEquals(1.0, result1NoUnit, 0);
+
+ double result1B = StorageUtil.getSizeInBytes("1B");
+ Assert.assertEquals(1.0, result1B, 0);
+
+ double result1BWithSpaces = StorageUtil.getSizeInBytes("1 B ");
+ Assert.assertEquals(1.0, result1BWithSpaces, 0);
+
+ double result1Kb = StorageUtil.getSizeInBytes("1KB");
+ Assert.assertEquals(1024.0, result1Kb, 0);
+
+ double result1KbWithSpaces = StorageUtil.getSizeInBytes(" 1 K B ");
+ Assert.assertEquals(1024.0, result1KbWithSpaces, 0);
+
+ double resultPoint5KB = StorageUtil.getSizeInBytes(".5KB");
+ Assert.assertEquals(512.0, resultPoint5KB, 0);
+
+ double resultPoint5SmallKB = StorageUtil.getSizeInBytes(".5kB");
+ Assert.assertEquals(512.0, resultPoint5SmallKB, 0);
+
+ double result1Mb = StorageUtil.getSizeInBytes("1MB");
+ Assert.assertEquals(1024.0 * 1024.0, result1Mb, 0);
+
+ double result1Point0Mb = StorageUtil.getSizeInBytes("1.0MB");
+ Assert.assertEquals(1024.0 * 1024.0, result1Point0Mb, 0);
+
+ double result01Point0Mb = StorageUtil.getSizeInBytes("01.0MB");
+ Assert.assertEquals(1024.0 * 1024.0, result01Point0Mb, 0);
+
+ // Invalid cases
+ invalidCase("");
+ invalidCase("99999999999999999999999999999999999999999999999999999999999999999999999999999999999999999");
+ invalidCase("32MB123");
+ invalidCase("1.1.1");
+ invalidCase("12KBMB");
+ invalidCase("MB");
+ invalidCase("1AB");
+ invalidCase("MB1MB");
+ invalidCase("123MBB");
+ }
+
+ private void invalidCase(String value) {
+ try {
+ StorageUtil.getSizeInBytes(value);
+ } catch (Exception ex) {
+ Assert.assertTrue(ex.toString()
+ .contains("IllegalArgumentException: The given string: " + value + " is not a byte unit string"));
+ }
+ }
+}
diff --git a/hyracks-fullstack/pom.xml b/hyracks-fullstack/pom.xml
index 9355f66..3645829 100644
--- a/hyracks-fullstack/pom.xml
+++ b/hyracks-fullstack/pom.xml
@@ -49,7 +49,8 @@
<root.dir>${basedir}</root.dir>
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
<file.encoding>UTF-8</file.encoding>
- <source.jdk.version>1.8</source.jdk.version>
+ <source.jdk.version>11</source.jdk.version>
+ <target.jdk.version>11</target.jdk.version>
<javac.xlint.value>all</javac.xlint.value>
<jvm.extraargs />
<sonar.jacoco.reportPath>${env.PWD}/target/jacoco-merged.exec</sonar.jacoco.reportPath>
@@ -150,6 +151,12 @@
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdfs</artifactId>
<version>${hadoop.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-all</artifactId>
+ </exclusion>
+ </exclusions>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
@@ -271,11 +278,6 @@
<version>${log4j.version}</version>
</dependency>
<dependency>
- <groupId>org.mockito</groupId>
- <artifactId>mockito-all</artifactId>
- <version>2.0.2-beta</version>
- </dependency>
- <dependency>
<groupId>javax.xml.bind</groupId>
<artifactId>jaxb-api</artifactId>
<version>2.3.0</version>
@@ -304,13 +306,23 @@
<groupId>org.apache.maven</groupId>
<artifactId>maven-core</artifactId>
<version>3.8.4</version>
- </dependency>
+ </dependency>
<dependency>
<groupId>org.apache.maven</groupId>
<artifactId>maven-plugin-api</artifactId>
<version>3.8.4</version>
</dependency>
<dependency>
+ <groupId>org.apache.maven.plugin-tools</groupId>
+ <artifactId>maven-plugin-annotations</artifactId>
+ <version>3.6.4</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-remote-resources-plugin</artifactId>
+ <version>1.7.0</version>
+ </dependency>
+ <dependency>
<groupId>it.unimi.dsi</groupId>
<artifactId>fastutil-core</artifactId>
<version>8.5.6</version>
@@ -326,6 +338,11 @@
<version>${log4j.version}</version>
</dependency>
<dependency>
+ <groupId>org.xerial.snappy</groupId>
+ <artifactId>snappy-java</artifactId>
+ <version>${snappy.version}</version>
+ </dependency>
+ <dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-core</artifactId>
<version>3.10.0</version>
@@ -345,11 +362,6 @@
<artifactId>powermock-core</artifactId>
<version>2.0.9</version>
</dependency>
- <dependency>
- <groupId>org.xerial.snappy</groupId>
- <artifactId>snappy-java</artifactId>
- <version>${snappy.version}</version>
- </dependency>
</dependencies>
</dependencyManagement>
<build>
@@ -511,10 +523,49 @@
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-enforcer-plugin</artifactId>
- <version>3.0.0-M1</version>
+ <artifactId>maven-antrun-plugin</artifactId>
<executions>
<execution>
+ <id>check-pom-packaging</id>
+ <phase>validate</phase>
+ <goals>
+ <goal>run</goal>
+ </goals>
+ <configuration>
+ <exportAntProperties>true</exportAntProperties>
+ <target xmlns:unless="ant:unless">
+ <condition property="skipPomEnforcement">
+ <not><equals arg1="${project.packaging}" arg2="pom"/></not>
+ </condition>
+ <echo message="will enforce non-existence of test source dir due to ${project.packaging} packaging" unless:set="skipPomEnforcement"/>
+ </target>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-enforcer-plugin</artifactId>
+ <executions>
+ <execution>
+ <id>ensure-no-tests-for-pom-packaging</id>
+ <phase>validate</phase>
+ <goals>
+ <goal>enforce</goal>
+ </goals>
+ <configuration>
+ <rules>
+ <requireFilesDontExist>
+ <files>
+ <file>${project.build.testSourceDirectory}</file>
+ </files>
+ <message>#### Tests cannot exist in projects with '${project.packaging}' packaging!</message>
+ </requireFilesDontExist>
+ </rules>
+ <skip>${skipPomEnforcement}</skip>
+ </configuration>
+ </execution>
+ <execution>
<id>enforce-versions</id>
<goals>
<goal>enforce</goal>
@@ -524,6 +575,20 @@
<requireMavenVersion>
<version>[3.3.9,)</version>
</requireMavenVersion>
+ <requireJavaVersion>
+ <version>[11,)</version>
+ </requireJavaVersion>
+ </rules>
+ </configuration>
+ </execution>
+ <execution>
+ <id>ensure-no-duplicate-deps</id>
+ <goals>
+ <goal>enforce</goal>
+ </goals>
+ <configuration>
+ <rules>
+ <banDuplicatePomDependencyVersions/>
</rules>
</configuration>
</execution>
@@ -720,13 +785,18 @@
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-dependency-plugin</artifactId>
- <version>3.1.1</version>
+ <version>3.1.2</version>
</plugin>
<plugin>
<groupId>org.apache.maven.doxia</groupId>
<artifactId>doxia-maven-plugin</artifactId>
<version>1.1.4</version>
</plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-enforcer-plugin</artifactId>
+ <version>3.0.0-M3</version>
+ </plugin>
</plugins>
</pluginManagement>
</build>
@@ -859,40 +929,6 @@
<failIfNoTests>false</failIfNoTests>
</properties>
</profile>
- <profile>
- <id>java8</id>
- <activation>
- <jdk>[1.8,)</jdk>
- </activation>
- <properties>
- <target.jdk.version>8</target.jdk.version>
- </properties>
- </profile>
- <profile>
- <id>java8-bootclasspath</id>
- <activation>
- <property>
- <name>java8-bootclasspath</name>
- </property>
- </activation>
- <properties>
- <target.jdk.version>8</target.jdk.version>
- </properties>
- <build>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-compiler-plugin</artifactId>
- <configuration>
- <compilerArgs combine.children="append">
- <arg>-bootclasspath</arg>
- <arg>${java8-bootclasspath}</arg>
- </compilerArgs>
- </configuration>
- </plugin>
- </plugins>
- </build>
- </profile>
</profiles>
<modules>
<module>hyracks</module>